From 4199da4a45cdda18ac8a5ab7fd779609c23157d0 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 26 Dec 2024 15:07:27 +0100 Subject: [PATCH 01/59] Add userspace routing --- client/firewall/iface.go | 4 + client/firewall/uspfilter/common/iface.go | 16 ++ .../firewall/uspfilter/forwarder/endpoint.go | 79 ++++++ .../firewall/uspfilter/forwarder/forwarder.go | 120 +++++++++ client/firewall/uspfilter/forwarder/tcp.go | 82 ++++++ client/firewall/uspfilter/forwarder/udp.go | 153 ++++++++++++ client/firewall/uspfilter/rule.go | 22 +- client/firewall/uspfilter/uspfilter.go | 236 ++++++++++++++---- client/firewall/uspfilter/uspfilter_test.go | 2 +- client/iface/device.go | 3 + client/iface/device/device_darwin.go | 5 + client/iface/device/device_kernel_unix.go | 6 + client/iface/device/device_netstack.go | 5 + client/iface/device/device_usp_unix.go | 5 + client/iface/device/device_windows.go | 5 + client/iface/iface.go | 7 + client/iface/iwginterface.go | 2 + client/iface/iwginterface_windows.go | 2 + client/internal/routemanager/manager.go | 8 +- go.mod | 2 +- go.sum | 2 - 21 files changed, 712 insertions(+), 54 deletions(-) create mode 100644 client/firewall/uspfilter/common/iface.go create mode 100644 client/firewall/uspfilter/forwarder/endpoint.go create mode 100644 client/firewall/uspfilter/forwarder/forwarder.go create mode 100644 client/firewall/uspfilter/forwarder/tcp.go create mode 100644 client/firewall/uspfilter/forwarder/udp.go diff --git a/client/firewall/iface.go b/client/firewall/iface.go index f349f9210a6..d842abaa124 100644 --- a/client/firewall/iface.go +++ b/client/firewall/iface.go @@ -1,6 +1,8 @@ package firewall import ( + wgdevice "golang.zx2c4.com/wireguard/device" + "github.com/netbirdio/netbird/client/iface/device" ) @@ -10,4 +12,6 @@ type IFaceMapper interface { Address() device.WGAddress IsUserspaceBind() bool SetFilter(device.PacketFilter) error + GetDevice() *device.FilteredDevice + GetWGDevice() *wgdevice.Device } diff --git a/client/firewall/uspfilter/common/iface.go b/client/firewall/uspfilter/common/iface.go new file mode 100644 index 00000000000..3bb12845773 --- /dev/null +++ b/client/firewall/uspfilter/common/iface.go @@ -0,0 +1,16 @@ +package common + +import ( + device2 "golang.zx2c4.com/wireguard/device" + + "github.com/netbirdio/netbird/client/iface" + "github.com/netbirdio/netbird/client/iface/device" +) + +// IFaceMapper defines subset methods of interface required for manager +type IFaceMapper interface { + SetFilter(device.PacketFilter) error + Address() iface.WGAddress + GetWGDevice() *device2.Device + GetDevice() *device.FilteredDevice +} diff --git a/client/firewall/uspfilter/forwarder/endpoint.go b/client/firewall/uspfilter/forwarder/endpoint.go new file mode 100644 index 00000000000..9f22fe3a20d --- /dev/null +++ b/client/firewall/uspfilter/forwarder/endpoint.go @@ -0,0 +1,79 @@ +package forwarder + +import ( + log "github.com/sirupsen/logrus" + wgdevice "golang.zx2c4.com/wireguard/device" + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/stack" +) + +// endpoint implements stack.LinkEndpoint and handles integration with the wireguard device +type endpoint struct { + dispatcher stack.NetworkDispatcher + device *wgdevice.Device + mtu uint32 +} + +func (e *endpoint) Attach(dispatcher stack.NetworkDispatcher) { + e.dispatcher = dispatcher +} + +func (e *endpoint) IsAttached() bool { + return e.dispatcher != nil +} + +func (e *endpoint) MTU() uint32 { + return e.mtu +} + +func (e *endpoint) Capabilities() stack.LinkEndpointCapabilities { + return stack.CapabilityNone +} + +func (e *endpoint) MaxHeaderLength() uint16 { + return 0 +} + +func (e *endpoint) LinkAddress() tcpip.LinkAddress { + return "" +} + +func (e *endpoint) WritePackets(pkts stack.PacketBufferList) (int, tcpip.Error) { + var written int + for _, pkt := range pkts.AsSlice() { + netHeader := header.IPv4(pkt.NetworkHeader().View().AsSlice()) + + data := stack.PayloadSince(pkt.NetworkHeader()) + if data == nil { + continue + } + + // Send the packet through WireGuard + address := netHeader.DestinationAddress() + + // TODO: handle dest ip addresses outside our network + err := e.device.CreateOutboundPacket(data.AsSlice(), address.AsSlice()) + if err != nil { + log.Errorf("CreateOutboundPacket: %v", err) + continue + } + written++ + } + + return written, nil +} + +func (e *endpoint) Wait() { +} + +func (e *endpoint) ARPHardwareType() header.ARPHardwareType { + return header.ARPHardwareNone +} + +func (e *endpoint) AddHeader(*stack.PacketBuffer) { +} + +func (e *endpoint) ParseHeader(*stack.PacketBuffer) bool { + return true +} diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go new file mode 100644 index 00000000000..4554ebb2014 --- /dev/null +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -0,0 +1,120 @@ +package forwarder + +import ( + "fmt" + + log "github.com/sirupsen/logrus" + "gvisor.dev/gvisor/pkg/buffer" + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/network/ipv4" + "gvisor.dev/gvisor/pkg/tcpip/stack" + "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" + "gvisor.dev/gvisor/pkg/tcpip/transport/udp" + + "github.com/netbirdio/netbird/client/firewall/uspfilter/common" +) + +const ( + receiveWindow = 32768 + maxInFlight = 1024 +) + +type Forwarder struct { + stack *stack.Stack + endpoint *endpoint + udpForwarder *udpForwarder +} + +func New(iface common.IFaceMapper) (*Forwarder, error) { + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{ + tcp.NewProtocol, + udp.NewProtocol, + }, + HandleLocal: false, + }) + + mtu, err := iface.GetDevice().MTU() + if err != nil { + return nil, fmt.Errorf("get MTU: %w", err) + } + nicID := tcpip.NICID(1) + endpoint := &endpoint{ + device: iface.GetWGDevice(), + mtu: uint32(mtu), + } + + if err := s.CreateNIC(nicID, endpoint); err != nil { + return nil, fmt.Errorf("failed to create NIC: %w", err) + } + + _, bits := iface.Address().Network.Mask.Size() + protoAddr := tcpip.ProtocolAddress{ + Protocol: ipv4.ProtocolNumber, + AddressWithPrefix: tcpip.AddressWithPrefix{ + Address: tcpip.AddrFromSlice(iface.Address().IP.To4()), + PrefixLen: bits, + }, + } + + if err := s.AddProtocolAddress(nicID, protoAddr, stack.AddressProperties{}); err != nil { + return nil, fmt.Errorf("failed to add protocol address: %w", err) + } + + defaultSubnet, err := tcpip.NewSubnet( + tcpip.AddrFrom4([4]byte{0, 0, 0, 0}), + tcpip.MaskFromBytes([]byte{0, 0, 0, 0}), + ) + if err != nil { + return nil, fmt.Errorf("creating default subnet: %w", err) + } + + if s.SetPromiscuousMode(nicID, true); err != nil { + return nil, fmt.Errorf("set promiscuous mode: %w", err) + } + if s.SetSpoofing(nicID, true); err != nil { + return nil, fmt.Errorf("set spoofing: %w", err) + } + + s.SetRouteTable([]tcpip.Route{ + { + Destination: defaultSubnet, + NIC: nicID, + }, + }) + + f := &Forwarder{ + stack: s, + endpoint: endpoint, + udpForwarder: newUDPForwarder(), + } + + // Set up TCP forwarder + tcpForwarder := tcp.NewForwarder(s, receiveWindow, maxInFlight, f.handleTCP) + s.SetTransportProtocolHandler(tcp.ProtocolNumber, tcpForwarder.HandlePacket) + + // Set up UDP forwarder + udpForwarder := udp.NewForwarder(s, f.handleUDP) + s.SetTransportProtocolHandler(udp.ProtocolNumber, udpForwarder.HandlePacket) + + log.Debugf("forwarder: Initialization complete with NIC %d", nicID) + return f, nil +} + +func (f *Forwarder) InjectIncomingPacket(payload []byte) error { + if len(payload) < header.IPv4MinimumSize { + return fmt.Errorf("packet too small: %d bytes", len(payload)) + } + + pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + Payload: buffer.MakeWithData(payload), + }) + defer pkt.DecRef() + + if f.endpoint.dispatcher != nil { + f.endpoint.dispatcher.DeliverNetworkPacket(ipv4.ProtocolNumber, pkt) + } + return nil +} diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go new file mode 100644 index 00000000000..4f406dea577 --- /dev/null +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -0,0 +1,82 @@ +package forwarder + +import ( + "fmt" + "io" + "net" + "sync" + + log "github.com/sirupsen/logrus" + "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" + "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" + "gvisor.dev/gvisor/pkg/waiter" +) + +// handleTCP is called by the TCP forwarder for new connections. +func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { + id := r.ID() + + dstAddr := id.LocalAddress + dstPort := id.LocalPort + dialAddr := fmt.Sprintf("%s:%d", dstAddr.String(), dstPort) + + // Dial the destination first + dialer := net.Dialer{} + outConn, err := dialer.Dial("tcp", dialAddr) + if err != nil { + r.Complete(true) + return + } + + // Create wait queue for blocking syscalls + wq := waiter.Queue{} + + ep, err2 := r.CreateEndpoint(&wq) + if err2 != nil { + if err := outConn.Close(); err != nil { + log.Errorf("forwarder: outConn close error: %v", err) + } + r.Complete(true) + return + } + + // Now that we've successfully connected to the destination, + // we can complete the incoming connection + r.Complete(false) + + inConn := gonet.NewTCPConn(&wq, ep) + + go f.proxyTCP(inConn, outConn) +} + +func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { + defer func() { + if err := inConn.Close(); err != nil { + log.Errorf("forwarder: inConn close error: %v", err) + } + if err := outConn.Close(); err != nil { + log.Errorf("forwarder: outConn close error: %v", err) + } + }() + + var wg sync.WaitGroup + wg.Add(2) + + go func() { + defer wg.Done() + _, err := io.Copy(outConn, inConn) + if err != nil { + log.Errorf("proxyTCP: copy error: %v", err) + } + }() + + go func() { + defer wg.Done() + _, err := io.Copy(inConn, outConn) + if err != nil { + log.Errorf("proxyTCP: copy error: %v", err) + } + }() + + wg.Wait() +} diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go new file mode 100644 index 00000000000..836d570cb91 --- /dev/null +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -0,0 +1,153 @@ +package forwarder + +import ( + "fmt" + "net" + "sync" + "time" + + log "github.com/sirupsen/logrus" + "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" + "gvisor.dev/gvisor/pkg/tcpip/transport/udp" + "gvisor.dev/gvisor/pkg/waiter" +) + +const ( + udpTimeout = 60 * time.Second +) + +type udpPacketConn struct { + conn *gonet.UDPConn + outConn net.Conn + lastTime time.Time +} + +type udpForwarder struct { + sync.RWMutex + conns map[string]*udpPacketConn +} + +func newUDPForwarder() *udpForwarder { + f := &udpForwarder{ + conns: make(map[string]*udpPacketConn), + } + go f.cleanup() + return f +} + +// cleanup periodically removes idle UDP connections +func (f *udpForwarder) cleanup() { + ticker := time.NewTicker(time.Minute) + defer ticker.Stop() + + for range ticker.C { + f.Lock() + now := time.Now() + for addr, conn := range f.conns { + if now.Sub(conn.lastTime) > udpTimeout { + conn.conn.Close() + conn.outConn.Close() + delete(f.conns, addr) + } + } + f.Unlock() + } +} + +// handleUDP is called by the UDP forwarder for new packets +func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { + id := r.ID() + dstAddr := fmt.Sprintf("%s:%d", id.LocalAddress.String(), id.LocalPort) + + // Create wait queue for blocking syscalls + wq := waiter.Queue{} + + ep, err := r.CreateEndpoint(&wq) + if err != nil { + log.Errorf("Create UDP endpoint error: %v", err) + return + } + + inConn := gonet.NewUDPConn(f.stack, &wq, ep) + + // Try to get existing connection or create a new one + f.udpForwarder.Lock() + pConn, exists := f.udpForwarder.conns[dstAddr] + if !exists { + outConn, err := net.Dial("udp", dstAddr) + if err != nil { + f.udpForwarder.Unlock() + if err := inConn.Close(); err != nil { + log.Errorf("forwader: UDP inConn close error: %v", err) + } + log.Errorf("forwarder> UDP dial error: %v", err) + return + } + + pConn = &udpPacketConn{ + conn: inConn, + outConn: outConn, + lastTime: time.Now(), + } + f.udpForwarder.conns[dstAddr] = pConn + + go f.proxyUDP(pConn, dstAddr) + } + f.udpForwarder.Unlock() +} + +func (f *Forwarder) proxyUDP(pConn *udpPacketConn, dstAddr string) { + defer func() { + if err := pConn.conn.Close(); err != nil { + log.Errorf("forwarder: inConn close error: %v", err) + } + if err := pConn.outConn.Close(); err != nil { + log.Errorf("forwarder: outConn close error: %v", err) + } + }() + + var wg sync.WaitGroup + wg.Add(2) + + // Handle outbound to inbound traffic + go func() { + defer wg.Done() + f.copyUDP(pConn.conn, pConn.outConn, dstAddr, "outbound->inbound") + }() + + // Handle inbound to outbound traffic + go func() { + defer wg.Done() + f.copyUDP(pConn.outConn, pConn.conn, dstAddr, "inbound->outbound") + }() + + wg.Wait() + + // Clean up the connection from the map + f.udpForwarder.Lock() + delete(f.udpForwarder.conns, dstAddr) + f.udpForwarder.Unlock() +} + +func (f *Forwarder) copyUDP(dst net.Conn, src net.Conn, dstAddr, direction string) { + buffer := make([]byte, 65535) + for { + n, err := src.Read(buffer) + if err != nil { + log.Errorf("UDP %s read error: %v", direction, err) + return + } + + _, err = dst.Write(buffer[:n]) + if err != nil { + log.Errorf("UDP %s write error: %v", direction, err) + continue + } + + f.udpForwarder.Lock() + if conn, ok := f.udpForwarder.conns[dstAddr]; ok { + conn.lastTime = time.Now() + } + f.udpForwarder.Unlock() + } +} diff --git a/client/firewall/uspfilter/rule.go b/client/firewall/uspfilter/rule.go index 5c1daccaf53..3d199ce6513 100644 --- a/client/firewall/uspfilter/rule.go +++ b/client/firewall/uspfilter/rule.go @@ -2,14 +2,15 @@ package uspfilter import ( "net" + "net/netip" "github.com/google/gopacket" firewall "github.com/netbirdio/netbird/client/firewall/manager" ) -// Rule to handle management of rules -type Rule struct { +// PeerRule to handle management of rules +type PeerRule struct { id string ip net.IP ipLayer gopacket.LayerType @@ -25,6 +26,21 @@ type Rule struct { } // GetRuleID returns the rule id -func (r *Rule) GetRuleID() string { +func (r *PeerRule) GetRuleID() string { + return r.id +} + +type RouteRule struct { + id string + sources []netip.Prefix + destination netip.Prefix + proto firewall.Protocol + srcPort *firewall.Port + dstPort *firewall.Port + action firewall.Action +} + +// GetRuleID returns the rule id +func (r *RouteRule) GetRuleID() string { return r.id } diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 24cfd6e9691..feed1887b44 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -14,9 +14,9 @@ import ( log "github.com/sirupsen/logrus" firewall "github.com/netbirdio/netbird/client/firewall/manager" + "github.com/netbirdio/netbird/client/firewall/uspfilter/common" "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" - "github.com/netbirdio/netbird/client/iface" - "github.com/netbirdio/netbird/client/iface/device" + "github.com/netbirdio/netbird/client/firewall/uspfilter/forwarder" "github.com/netbirdio/netbird/client/internal/statemanager" ) @@ -24,34 +24,34 @@ const layerTypeAll = 0 const EnvDisableConntrack = "NB_DISABLE_CONNTRACK" +// TODO: Add env var to disable routing + var ( errRouteNotSupported = fmt.Errorf("route not supported with userspace firewall") ) -// IFaceMapper defines subset methods of interface required for manager -type IFaceMapper interface { - SetFilter(device.PacketFilter) error - Address() iface.WGAddress -} - // RuleSet is a set of rules grouped by a string key -type RuleSet map[string]Rule +type RuleSet map[string]PeerRule // Manager userspace firewall manager type Manager struct { outgoingRules map[string]RuleSet incomingRules map[string]RuleSet + routeRules map[string]RouteRule wgNetwork *net.IPNet decoders sync.Pool - wgIface IFaceMapper + wgIface common.IFaceMapper nativeFirewall firewall.Manager mutex sync.RWMutex + routingEnabled bool + stateful bool udpTracker *conntrack.UDPTracker icmpTracker *conntrack.ICMPTracker tcpTracker *conntrack.TCPTracker + forwarder *forwarder.Forwarder } // decoder for packages @@ -68,11 +68,11 @@ type decoder struct { } // Create userspace firewall manager constructor -func Create(iface IFaceMapper) (*Manager, error) { +func Create(iface common.IFaceMapper) (*Manager, error) { return create(iface) } -func CreateWithNativeFirewall(iface IFaceMapper, nativeFirewall firewall.Manager) (*Manager, error) { +func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall.Manager) (*Manager, error) { mgr, err := create(iface) if err != nil { return nil, err @@ -82,7 +82,7 @@ func CreateWithNativeFirewall(iface IFaceMapper, nativeFirewall firewall.Manager return mgr, nil } -func create(iface IFaceMapper) (*Manager, error) { +func create(iface common.IFaceMapper) (*Manager, error) { disableConntrack, _ := strconv.ParseBool(os.Getenv(EnvDisableConntrack)) m := &Manager{ @@ -101,8 +101,11 @@ func create(iface IFaceMapper) (*Manager, error) { }, outgoingRules: make(map[string]RuleSet), incomingRules: make(map[string]RuleSet), + routeRules: make(map[string]RouteRule), wgIface: iface, stateful: !disableConntrack, + // TODO: fix + routingEnabled: true, } // Only initialize trackers if stateful mode is enabled @@ -114,8 +117,23 @@ func create(iface IFaceMapper) (*Manager, error) { m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout) } + intf := iface.GetWGDevice() + if intf == nil { + log.Info("forwarding not supported") + // Only supported in userspace mode as we need to inject packets back into wireguard directly + // TODO: Check if native firewall can do the job, in that case just forward everything (restores previous behavior) + m.routingEnabled = false + } else { + var err error + m.forwarder, err = forwarder.New(iface) + if err != nil { + log.Errorf("failed to create forwarder: %v", err) + m.routingEnabled = false + } + } + if err := iface.SetFilter(m); err != nil { - return nil, err + return nil, fmt.Errorf("set filter: %w", err) } return m, nil } @@ -161,7 +179,7 @@ func (m *Manager) AddPeerFiltering( ipsetName string, comment string, ) ([]firewall.Rule, error) { - r := Rule{ + r := PeerRule{ id: uuid.New().String(), ip: ip, ipLayer: layers.LayerTypeIPv6, @@ -217,18 +235,44 @@ func (m *Manager) AddPeerFiltering( return []firewall.Rule{&r}, nil } -func (m *Manager) AddRouteFiltering(sources []netip.Prefix, destination netip.Prefix, proto firewall.Protocol, sPort *firewall.Port, dPort *firewall.Port, action firewall.Action) (firewall.Rule, error) { - if m.nativeFirewall == nil { - return nil, errRouteNotSupported +func (m *Manager) AddRouteFiltering( + sources []netip.Prefix, + destination netip.Prefix, + proto firewall.Protocol, + sPort *firewall.Port, + dPort *firewall.Port, + action firewall.Action, +) (firewall.Rule, error) { + m.mutex.Lock() + defer m.mutex.Unlock() + + ruleID := uuid.New().String() + rule := RouteRule{ + id: ruleID, + sources: sources, + destination: destination, + proto: proto, + srcPort: sPort, + dstPort: dPort, + action: action, } - return m.nativeFirewall.AddRouteFiltering(sources, destination, proto, sPort, dPort, action) + + m.routeRules[ruleID] = rule + + return &rule, nil } func (m *Manager) DeleteRouteRule(rule firewall.Rule) error { - if m.nativeFirewall == nil { - return errRouteNotSupported + m.mutex.Lock() + defer m.mutex.Unlock() + + ruleID := rule.GetRuleID() + if _, exists := m.routeRules[ruleID]; !exists { + return fmt.Errorf("route rule not found: %s", ruleID) } - return m.nativeFirewall.DeleteRouteRule(rule) + + delete(m.routeRules, ruleID) + return nil } // DeletePeerRule from the firewall by rule definition @@ -236,7 +280,7 @@ func (m *Manager) DeletePeerRule(rule firewall.Rule) error { m.mutex.Lock() defer m.mutex.Unlock() - r, ok := rule.(*Rule) + r, ok := rule.(*PeerRule) if !ok { return fmt.Errorf("delete rule: invalid rule type: %T", rule) } @@ -279,7 +323,11 @@ func (m *Manager) DropIncoming(packetData []byte) bool { return m.dropFilter(packetData, m.incomingRules) } -// processOutgoingHooks processes UDP hooks for outgoing packets and tracks TCP/UDP/ICMP +func (m *Manager) isLocalIP(ip net.IP) bool { + // TODO: add other interface IPs and keep track of them + return ip.Equal(m.wgIface.Address().IP) +} + func (m *Manager) processOutgoingHooks(packetData []byte) bool { m.mutex.RLock() defer m.mutex.RUnlock() @@ -300,18 +348,11 @@ func (m *Manager) processOutgoingHooks(packetData []byte) bool { return false } - // Always process UDP hooks - if d.decoded[1] == layers.LayerTypeUDP { - // Track UDP state only if enabled - if m.stateful { - m.trackUDPOutbound(d, srcIP, dstIP) - } - return m.checkUDPHooks(d, dstIP, packetData) - } - - // Track other protocols only if stateful mode is enabled + // Track all protocols if stateful mode is enabled if m.stateful { switch d.decoded[1] { + case layers.LayerTypeUDP: + m.trackUDPOutbound(d, srcIP, dstIP) case layers.LayerTypeTCP: m.trackTCPOutbound(d, srcIP, dstIP) case layers.LayerTypeICMPv4: @@ -319,6 +360,11 @@ func (m *Manager) processOutgoingHooks(packetData []byte) bool { } } + // Process UDP hooks even if stateful mode is disabled + if d.decoded[1] == layers.LayerTypeUDP { + return m.checkUDPHooks(d, dstIP, packetData) + } + return false } @@ -409,6 +455,7 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { defer m.decoders.Put(d) if !m.isValidPacket(d, packetData) { + log.Debugf("invalid packet: %v", d.decoded) return true } @@ -418,16 +465,69 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { return true } - if !m.isWireguardTraffic(srcIP, dstIP) { - return false - } + // Check if this is local or routed traffic + isLocal := m.isLocalIP(dstIP) - // Check connection state only if enabled + // For all inbound traffic, first check if it matches a tracked connection. + // This must happen before any other filtering because the packets are statefully tracked. if m.stateful && m.isValidTrackedConnection(d, srcIP, dstIP) { return false } - return m.applyRules(srcIP, packetData, rules, d) + // Handle local traffic - apply peer ACLs + if isLocal { + return m.applyRules(srcIP, packetData, rules, d) + } + + // Handle routed traffic + // TODO: Handle replies for [routed network -> netbird peer], we don't need to start the forwarder here + // We might need to apply NAT + // Don't handle routing if not enabled + if !m.routingEnabled { + return true + } + + // Get protocol and ports for route ACL check + proto := getProtocolFromPacket(d) + srcPort, dstPort := getPortsFromPacket(d) + + // Check route ACLs + if !m.checkRouteACLs(srcIP, dstIP, proto, srcPort, dstPort) { + return true + } + + // Let forwarder handle the packet if it passed route ACLs + err := m.forwarder.InjectIncomingPacket(packetData) + if err != nil { + log.Errorf("Failed to inject incoming packet: %v", err) + } + + // Default: drop + return true +} + +func getProtocolFromPacket(d *decoder) firewall.Protocol { + switch d.decoded[1] { + case layers.LayerTypeTCP: + return firewall.ProtocolTCP + case layers.LayerTypeUDP: + return firewall.ProtocolUDP + case layers.LayerTypeICMPv4, layers.LayerTypeICMPv6: + return firewall.ProtocolICMP + default: + return firewall.ProtocolALL + } +} + +func getPortsFromPacket(d *decoder) (srcPort, dstPort uint16) { + switch d.decoded[1] { + case layers.LayerTypeTCP: + return uint16(d.tcp.SrcPort), uint16(d.tcp.DstPort) + case layers.LayerTypeUDP: + return uint16(d.udp.SrcPort), uint16(d.udp.DstPort) + default: + return 0, 0 + } } func (m *Manager) isValidPacket(d *decoder, packetData []byte) bool { @@ -498,7 +598,7 @@ func (m *Manager) applyRules(srcIP net.IP, packetData []byte, rules map[string]R return true } -func validateRule(ip net.IP, packetData []byte, rules map[string]Rule, d *decoder) (bool, bool) { +func validateRule(ip net.IP, packetData []byte, rules map[string]PeerRule, d *decoder) (bool, bool) { payloadLayer := d.decoded[1] for _, rule := range rules { if rule.matchByIP && !ip.Equal(rule.ip) { @@ -547,6 +647,56 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]Rule, d *decode return false, false } +func (m *Manager) checkRouteACLs(srcIP, dstIP net.IP, proto firewall.Protocol, srcPort, dstPort uint16) bool { + m.mutex.RLock() + defer m.mutex.RUnlock() + + srcAddr, _ := netip.AddrFromSlice(srcIP) + dstAddr, _ := netip.AddrFromSlice(dstIP) + + // Default deny if no rules match + matched := false + + for _, rule := range m.routeRules { + // Check destination + if !rule.destination.Contains(dstAddr) { + continue + } + + // Check if source matches any source prefix + sourceMatched := false + for _, src := range rule.sources { + if src.Contains(srcAddr) { + sourceMatched = true + break + } + } + if !sourceMatched { + continue + } + + // Check protocol + if rule.proto != firewall.ProtocolALL && rule.proto != proto { + continue + } + + // Check ports if specified + if rule.srcPort != nil && rule.srcPort.Values[0] != int(srcPort) { + continue + } + if rule.dstPort != nil && rule.dstPort.Values[0] != int(dstPort) { + continue + } + + matched = true + if rule.action == firewall.ActionDrop { + return false + } + } + + return matched +} + // SetNetwork of the wireguard interface to which filtering applied func (m *Manager) SetNetwork(network *net.IPNet) { m.wgNetwork = network @@ -558,7 +708,7 @@ func (m *Manager) SetNetwork(network *net.IPNet) { func (m *Manager) AddUDPPacketHook( in bool, ip net.IP, dPort uint16, hook func([]byte) bool, ) string { - r := Rule{ + r := PeerRule{ id: uuid.New().String(), ip: ip, protoLayer: layers.LayerTypeUDP, @@ -577,12 +727,12 @@ func (m *Manager) AddUDPPacketHook( if in { r.direction = firewall.RuleDirectionIN if _, ok := m.incomingRules[r.ip.String()]; !ok { - m.incomingRules[r.ip.String()] = make(map[string]Rule) + m.incomingRules[r.ip.String()] = make(map[string]PeerRule) } m.incomingRules[r.ip.String()][r.id] = r } else { if _, ok := m.outgoingRules[r.ip.String()]; !ok { - m.outgoingRules[r.ip.String()] = make(map[string]Rule) + m.outgoingRules[r.ip.String()] = make(map[string]PeerRule) } m.outgoingRules[r.ip.String()][r.id] = r } diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index d3563e6f251..443d82607ea 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -194,7 +194,7 @@ func TestAddUDPPacketHook(t *testing.T) { manager.AddUDPPacketHook(tt.in, tt.ip, tt.dPort, tt.hook) - var addedRule Rule + var addedRule PeerRule if tt.in { if len(manager.incomingRules[tt.ip.String()]) != 1 { t.Errorf("expected 1 incoming rule, got %d", len(manager.incomingRules)) diff --git a/client/iface/device.go b/client/iface/device.go index 0d4e6914554..2a170adfb41 100644 --- a/client/iface/device.go +++ b/client/iface/device.go @@ -3,6 +3,8 @@ package iface import ( + wgdevice "golang.zx2c4.com/wireguard/device" + "github.com/netbirdio/netbird/client/iface/bind" "github.com/netbirdio/netbird/client/iface/device" ) @@ -15,4 +17,5 @@ type WGTunDevice interface { DeviceName() string Close() error FilteredDevice() *device.FilteredDevice + Device() *wgdevice.Device } diff --git a/client/iface/device/device_darwin.go b/client/iface/device/device_darwin.go index b5a128bc1cc..fe7ed175207 100644 --- a/client/iface/device/device_darwin.go +++ b/client/iface/device/device_darwin.go @@ -117,6 +117,11 @@ func (t *TunDevice) FilteredDevice() *FilteredDevice { return t.filteredDevice } +// Device returns the wireguard device +func (t *TunDevice) Device() *device.Device { + return t.device +} + // assignAddr Adds IP address to the tunnel interface and network route based on the range provided func (t *TunDevice) assignAddr() error { cmd := exec.Command("ifconfig", t.name, "inet", t.address.IP.String(), t.address.IP.String()) diff --git a/client/iface/device/device_kernel_unix.go b/client/iface/device/device_kernel_unix.go index f355d2cf76a..978e72b79ec 100644 --- a/client/iface/device/device_kernel_unix.go +++ b/client/iface/device/device_kernel_unix.go @@ -9,6 +9,7 @@ import ( "github.com/pion/transport/v3" log "github.com/sirupsen/logrus" + "golang.zx2c4.com/wireguard/device" "github.com/netbirdio/netbird/client/iface/bind" "github.com/netbirdio/netbird/client/iface/configurer" @@ -153,6 +154,11 @@ func (t *TunKernelDevice) DeviceName() string { return t.name } +// Device returns the wireguard device, not applicable for kernel devices +func (t *TunKernelDevice) Device() *device.Device { + return nil +} + func (t *TunKernelDevice) FilteredDevice() *FilteredDevice { return nil } diff --git a/client/iface/device/device_netstack.go b/client/iface/device/device_netstack.go index f5d39e9e074..c7d297187ed 100644 --- a/client/iface/device/device_netstack.go +++ b/client/iface/device/device_netstack.go @@ -117,3 +117,8 @@ func (t *TunNetstackDevice) DeviceName() string { func (t *TunNetstackDevice) FilteredDevice() *FilteredDevice { return t.filteredDevice } + +// Device returns the wireguard device +func (t *TunNetstackDevice) Device() *device.Device { + return t.device +} diff --git a/client/iface/device/device_usp_unix.go b/client/iface/device/device_usp_unix.go index 643d77565c2..1a154501a6c 100644 --- a/client/iface/device/device_usp_unix.go +++ b/client/iface/device/device_usp_unix.go @@ -128,6 +128,11 @@ func (t *USPDevice) FilteredDevice() *FilteredDevice { return t.filteredDevice } +// Device returns the wireguard device +func (t *USPDevice) Device() *device.Device { + return t.device +} + // assignAddr Adds IP address to the tunnel interface func (t *USPDevice) assignAddr() error { link := newWGLink(t.name) diff --git a/client/iface/device/device_windows.go b/client/iface/device/device_windows.go index 86968d06d7e..e603d7696f9 100644 --- a/client/iface/device/device_windows.go +++ b/client/iface/device/device_windows.go @@ -150,6 +150,11 @@ func (t *TunDevice) FilteredDevice() *FilteredDevice { return t.filteredDevice } +// Device returns the wireguard device +func (t *TunDevice) Device() *device.Device { + return t.device +} + func (t *TunDevice) GetInterfaceGUIDString() (string, error) { if t.nativeTunDevice == nil { return "", fmt.Errorf("interface has not been initialized yet") diff --git a/client/iface/iface.go b/client/iface/iface.go index 1fb9c269179..64219975f5d 100644 --- a/client/iface/iface.go +++ b/client/iface/iface.go @@ -11,6 +11,8 @@ import ( log "github.com/sirupsen/logrus" "golang.zx2c4.com/wireguard/wgctrl/wgtypes" + wgdevice "golang.zx2c4.com/wireguard/device" + "github.com/netbirdio/netbird/client/errors" "github.com/netbirdio/netbird/client/iface/bind" "github.com/netbirdio/netbird/client/iface/configurer" @@ -203,6 +205,11 @@ func (w *WGIface) GetDevice() *device.FilteredDevice { return w.tun.FilteredDevice() } +// GetWGDevice returns the WireGuard device +func (w *WGIface) GetWGDevice() *wgdevice.Device { + return w.tun.Device() +} + // GetStats returns the last handshake time, rx and tx bytes for the given peer func (w *WGIface) GetStats(peerKey string) (configurer.WGStats, error) { return w.configurer.GetStats(peerKey) diff --git a/client/iface/iwginterface.go b/client/iface/iwginterface.go index f5ab2953905..472ab45f9d8 100644 --- a/client/iface/iwginterface.go +++ b/client/iface/iwginterface.go @@ -6,6 +6,7 @@ import ( "net" "time" + wgdevice "golang.zx2c4.com/wireguard/device" "golang.zx2c4.com/wireguard/wgctrl/wgtypes" "github.com/netbirdio/netbird/client/iface/bind" @@ -32,5 +33,6 @@ type IWGIface interface { SetFilter(filter device.PacketFilter) error GetFilter() device.PacketFilter GetDevice() *device.FilteredDevice + GetWGDevice() *wgdevice.Device GetStats(peerKey string) (configurer.WGStats, error) } diff --git a/client/iface/iwginterface_windows.go b/client/iface/iwginterface_windows.go index 96eec52a502..c9183cafdce 100644 --- a/client/iface/iwginterface_windows.go +++ b/client/iface/iwginterface_windows.go @@ -4,6 +4,7 @@ import ( "net" "time" + wgdevice "golang.zx2c4.com/wireguard/device" "golang.zx2c4.com/wireguard/wgctrl/wgtypes" "github.com/netbirdio/netbird/client/iface/bind" @@ -30,6 +31,7 @@ type IWGIface interface { SetFilter(filter device.PacketFilter) error GetFilter() device.PacketFilter GetDevice() *device.FilteredDevice + GetWGDevice() *wgdevice.Device GetStats(peerKey string) (configurer.WGStats, error) GetInterfaceGUIDString() (string, error) } diff --git a/client/internal/routemanager/manager.go b/client/internal/routemanager/manager.go index 389e97e2dcc..0d4a21ac461 100644 --- a/client/internal/routemanager/manager.go +++ b/client/internal/routemanager/manager.go @@ -383,10 +383,10 @@ func (m *DefaultManager) classifyRoutes(newRoutes []*route.Route) (map[route.ID] if newRoute.Peer == m.pubKey { ownNetworkIDs[haID] = true // only linux is supported for now - if runtime.GOOS != "linux" { - log.Warnf("received a route to manage, but agent doesn't support router mode on %s OS", runtime.GOOS) - continue - } + //if runtime.GOOS != "linux" { + // log.Warnf("received a route to manage, but agent doesn't support router mode on %s OS", runtime.GOOS) + // continue + //} newServerRoutesMap[newRoute.ID] = newRoute } } diff --git a/go.mod b/go.mod index d48280df02a..b1da755121d 100644 --- a/go.mod +++ b/go.mod @@ -99,6 +99,7 @@ require ( gorm.io/driver/postgres v1.5.7 gorm.io/driver/sqlite v1.5.3 gorm.io/gorm v1.25.7-0.20240204074919-46816ad31dde + gvisor.dev/gvisor v0.0.0-20231020174304-b8a429915ff1 nhooyr.io/websocket v1.8.11 ) @@ -229,7 +230,6 @@ require ( gopkg.in/square/go-jose.v2 v2.6.0 // indirect gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637 // indirect - gvisor.dev/gvisor v0.0.0-20231020174304-b8a429915ff1 // indirect k8s.io/apimachinery v0.26.2 // indirect ) diff --git a/go.sum b/go.sum index 540cbf20bb9..5511fbe3049 100644 --- a/go.sum +++ b/go.sum @@ -527,8 +527,6 @@ github.com/netbirdio/service v0.0.0-20240911161631-f62744f42502 h1:3tHlFmhTdX9ax github.com/netbirdio/service v0.0.0-20240911161631-f62744f42502/go.mod h1:CIMRFEJVL+0DS1a3Nx06NaMn4Dz63Ng6O7dl0qH0zVM= github.com/netbirdio/signal-dispatcher/dispatcher v0.0.0-20241010133937-e0df50df217d h1:bRq5TKgC7Iq20pDiuC54yXaWnAVeS5PdGpSokFTlR28= github.com/netbirdio/signal-dispatcher/dispatcher v0.0.0-20241010133937-e0df50df217d/go.mod h1:5/sjFmLb8O96B5737VCqhHyGRzNFIaN/Bu7ZodXc3qQ= -github.com/netbirdio/wireguard-go v0.0.0-20241125150134-f9cdce5e32e9 h1:Pu/7EukijT09ynHUOzQYW7cC3M/BKU8O4qyN/TvTGoY= -github.com/netbirdio/wireguard-go v0.0.0-20241125150134-f9cdce5e32e9/go.mod h1:tkCQ4FQXmpAgYVh++1cq16/dH4QJtmvpRv19DWGAHSA= github.com/nicksnyder/go-i18n/v2 v2.4.0 h1:3IcvPOAvnCKwNm0TB0dLDTuawWEj+ax/RERNC+diLMM= github.com/nicksnyder/go-i18n/v2 v2.4.0/go.mod h1:nxYSZE9M0bf3Y70gPQjN9ha7XNHX7gMc814+6wVyEI4= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= From b43a8c56df922a4768df8b4d74ccde1096b2b2c7 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 30 Dec 2024 01:52:52 +0100 Subject: [PATCH 02/59] Update wireguard-go ref --- go.mod | 2 +- go.sum | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index b1da755121d..8b141028350 100644 --- a/go.mod +++ b/go.mod @@ -237,7 +237,7 @@ replace github.com/kardianos/service => github.com/netbirdio/service v0.0.0-2024 replace github.com/getlantern/systray => github.com/netbirdio/systray v0.0.0-20231030152038-ef1ed2a27949 -replace golang.zx2c4.com/wireguard => github.com/netbirdio/wireguard-go v0.0.0-20241125150134-f9cdce5e32e9 +replace golang.zx2c4.com/wireguard => github.com/netbirdio/wireguard-go v0.0.0-20241230120307-6a676aebaaf6 replace github.com/cloudflare/circl => github.com/cunicu/circl v0.0.0-20230801113412-fec58fc7b5f6 diff --git a/go.sum b/go.sum index 5511fbe3049..ffa17aed4ac 100644 --- a/go.sum +++ b/go.sum @@ -527,6 +527,8 @@ github.com/netbirdio/service v0.0.0-20240911161631-f62744f42502 h1:3tHlFmhTdX9ax github.com/netbirdio/service v0.0.0-20240911161631-f62744f42502/go.mod h1:CIMRFEJVL+0DS1a3Nx06NaMn4Dz63Ng6O7dl0qH0zVM= github.com/netbirdio/signal-dispatcher/dispatcher v0.0.0-20241010133937-e0df50df217d h1:bRq5TKgC7Iq20pDiuC54yXaWnAVeS5PdGpSokFTlR28= github.com/netbirdio/signal-dispatcher/dispatcher v0.0.0-20241010133937-e0df50df217d/go.mod h1:5/sjFmLb8O96B5737VCqhHyGRzNFIaN/Bu7ZodXc3qQ= +github.com/netbirdio/wireguard-go v0.0.0-20241230120307-6a676aebaaf6 h1:X5h5QgP7uHAv78FWgHV8+WYLjHxK9v3ilkVXT1cpCrQ= +github.com/netbirdio/wireguard-go v0.0.0-20241230120307-6a676aebaaf6/go.mod h1:tkCQ4FQXmpAgYVh++1cq16/dH4QJtmvpRv19DWGAHSA= github.com/nicksnyder/go-i18n/v2 v2.4.0 h1:3IcvPOAvnCKwNm0TB0dLDTuawWEj+ax/RERNC+diLMM= github.com/nicksnyder/go-i18n/v2 v2.4.0/go.mod h1:nxYSZE9M0bf3Y70gPQjN9ha7XNHX7gMc814+6wVyEI4= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= From fad82ee65c969aafc8214756a7447dce750ab847 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 30 Dec 2024 13:34:51 +0100 Subject: [PATCH 03/59] Add stop methods and improve udp implementation --- client/firewall/uspfilter/allow_netbird.go | 4 + .../uspfilter/allow_netbird_windows.go | 4 + .../firewall/uspfilter/forwarder/forwarder.go | 21 ++ client/firewall/uspfilter/forwarder/tcp.go | 42 ++-- client/firewall/uspfilter/forwarder/udp.go | 193 +++++++++++++----- 5 files changed, 193 insertions(+), 71 deletions(-) diff --git a/client/firewall/uspfilter/allow_netbird.go b/client/firewall/uspfilter/allow_netbird.go index cc07922559d..29709509011 100644 --- a/client/firewall/uspfilter/allow_netbird.go +++ b/client/firewall/uspfilter/allow_netbird.go @@ -30,6 +30,10 @@ func (m *Manager) Reset(stateManager *statemanager.Manager) error { m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout) } + if m.forwarder != nil { + m.forwarder.Stop() + } + if m.nativeFirewall != nil { return m.nativeFirewall.Reset(stateManager) } diff --git a/client/firewall/uspfilter/allow_netbird_windows.go b/client/firewall/uspfilter/allow_netbird_windows.go index 0d55d62689c..42bf0896ec4 100644 --- a/client/firewall/uspfilter/allow_netbird_windows.go +++ b/client/firewall/uspfilter/allow_netbird_windows.go @@ -42,6 +42,10 @@ func (m *Manager) Reset(*statemanager.Manager) error { m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout) } + if m.forwarder != nil { + m.forwarder.Stop() + } + if !isWindowsFirewallReachable() { return nil } diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index 4554ebb2014..815c7da09c6 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -1,6 +1,7 @@ package forwarder import ( + "context" "fmt" log "github.com/sirupsen/logrus" @@ -24,9 +25,12 @@ type Forwarder struct { stack *stack.Stack endpoint *endpoint udpForwarder *udpForwarder + ctx context.Context + cancel context.CancelFunc } func New(iface common.IFaceMapper) (*Forwarder, error) { + s := stack.New(stack.Options{ NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, TransportProtocols: []stack.TransportProtocolFactory{ @@ -85,10 +89,13 @@ func New(iface common.IFaceMapper) (*Forwarder, error) { }, }) + ctx, cancel := context.WithCancel(context.Background()) f := &Forwarder{ stack: s, endpoint: endpoint, udpForwarder: newUDPForwarder(), + ctx: ctx, + cancel: cancel, } // Set up TCP forwarder @@ -118,3 +125,17 @@ func (f *Forwarder) InjectIncomingPacket(payload []byte) error { } return nil } + +// Stop gracefully shuts down the forwarder +func (f *Forwarder) Stop() error { + f.cancel() + + if f.udpForwarder != nil { + f.udpForwarder.Stop() + } + + f.stack.Close() + f.stack.Wait() + + return nil +} diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index 4f406dea577..90967b6f57f 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -1,10 +1,10 @@ package forwarder import ( + "context" "fmt" "io" "net" - "sync" log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" @@ -20,9 +20,7 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { dstPort := id.LocalPort dialAddr := fmt.Sprintf("%s:%d", dstAddr.String(), dstPort) - // Dial the destination first - dialer := net.Dialer{} - outConn, err := dialer.Dial("tcp", dialAddr) + outConn, err := (&net.Dialer{}).DialContext(f.ctx, "tcp", dialAddr) if err != nil { r.Complete(true) return @@ -40,8 +38,7 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { return } - // Now that we've successfully connected to the destination, - // we can complete the incoming connection + // Complete the handshake r.Complete(false) inConn := gonet.NewTCPConn(&wq, ep) @@ -59,24 +56,35 @@ func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { } }() - var wg sync.WaitGroup - wg.Add(2) + // Create context for managing the proxy goroutines + ctx, cancel := context.WithCancel(f.ctx) + defer cancel() + + errChan := make(chan error, 2) go func() { - defer wg.Done() - _, err := io.Copy(outConn, inConn) - if err != nil { - log.Errorf("proxyTCP: copy error: %v", err) + n, err := io.Copy(outConn, inConn) + if err != nil && !isClosedError(err) { + log.Errorf("proxyTCP: inbound->outbound copy error after %d bytes: %v", n, err) } + errChan <- err }() go func() { - defer wg.Done() - _, err := io.Copy(inConn, outConn) - if err != nil { - log.Errorf("proxyTCP: copy error: %v", err) + n, err := io.Copy(inConn, outConn) + if err != nil && !isClosedError(err) { + log.Errorf("proxyTCP: outbound->inbound copy error after %d bytes: %v", n, err) } + errChan <- err }() - wg.Wait() + select { + case <-ctx.Done(): + return + case err := <-errChan: + if err != nil && !isClosedError(err) { + log.Errorf("proxyTCP: copy error: %v", err) + } + return + } } diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 836d570cb91..7d201024eb0 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -1,6 +1,8 @@ package forwarder import ( + "context" + "errors" "fmt" "net" "sync" @@ -8,49 +10,94 @@ import ( log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" + "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" "gvisor.dev/gvisor/pkg/waiter" ) const ( - udpTimeout = 60 * time.Second + udpTimeout = 60 * time.Second + maxPacketSize = 65535 ) type udpPacketConn struct { conn *gonet.UDPConn outConn net.Conn lastTime time.Time + cancel context.CancelFunc } type udpForwarder struct { sync.RWMutex - conns map[string]*udpPacketConn + conns map[stack.TransportEndpointID]*udpPacketConn + bufPool sync.Pool + ctx context.Context + cancel context.CancelFunc } func newUDPForwarder() *udpForwarder { + ctx, cancel := context.WithCancel(context.Background()) f := &udpForwarder{ - conns: make(map[string]*udpPacketConn), + conns: make(map[stack.TransportEndpointID]*udpPacketConn), + ctx: ctx, + cancel: cancel, + bufPool: sync.Pool{ + New: func() any { + b := make([]byte, maxPacketSize) + return &b + }, + }, } go f.cleanup() return f } +// Stop stops the UDP forwarder and all active connections +func (f *udpForwarder) Stop() { + f.cancel() + + f.Lock() + defer f.Unlock() + + for id, conn := range f.conns { + conn.cancel() + if err := conn.conn.Close(); err != nil { + log.Errorf("forwarder: UDP conn close error for %v: %v", id, err) + } + if err := conn.outConn.Close(); err != nil { + log.Errorf("forwarder: UDP outConn close error for %v: %v", id, err) + } + delete(f.conns, id) + } +} + // cleanup periodically removes idle UDP connections func (f *udpForwarder) cleanup() { ticker := time.NewTicker(time.Minute) defer ticker.Stop() - for range ticker.C { - f.Lock() - now := time.Now() - for addr, conn := range f.conns { - if now.Sub(conn.lastTime) > udpTimeout { - conn.conn.Close() - conn.outConn.Close() - delete(f.conns, addr) + for { + select { + case <-f.ctx.Done(): + return + case <-ticker.C: + f.Lock() + now := time.Now() + for id, conn := range f.conns { + if now.Sub(conn.lastTime) > udpTimeout { + conn.cancel() + if err := conn.conn.Close(); err != nil { + log.Errorf("forwarder: UDP conn close error for %v: %v", id, err) + } + if err := conn.outConn.Close(); err != nil { + log.Errorf("forwarder: UDP outConn close error for %v: %v", id, err) + } + delete(f.conns, id) + log.Debugf("forwarder: cleaned up idle UDP connection %v", id) + } } + f.Unlock() } - f.Unlock() } } @@ -59,12 +106,17 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { id := r.ID() dstAddr := fmt.Sprintf("%s:%d", id.LocalAddress.String(), id.LocalPort) + if f.ctx.Err() != nil { + log.Debug("forwarder: context done, dropping UDP packet") + return + } + // Create wait queue for blocking syscalls wq := waiter.Queue{} ep, err := r.CreateEndpoint(&wq) if err != nil { - log.Errorf("Create UDP endpoint error: %v", err) + log.Errorf("forwarder: failed to create UDP endpoint: %v", err) return } @@ -72,82 +124,115 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { // Try to get existing connection or create a new one f.udpForwarder.Lock() - pConn, exists := f.udpForwarder.conns[dstAddr] + defer f.udpForwarder.Unlock() + + pConn, exists := f.udpForwarder.conns[id] if !exists { - outConn, err := net.Dial("udp", dstAddr) + outConn, err := (&net.Dialer{}).DialContext(f.ctx, "udp", dstAddr) if err != nil { - f.udpForwarder.Unlock() if err := inConn.Close(); err != nil { - log.Errorf("forwader: UDP inConn close error: %v", err) + log.Errorf("forwarder: UDP inConn close error for %v: %v", id, err) } - log.Errorf("forwarder> UDP dial error: %v", err) + log.Errorf("forwarder: UDP dial error for %v: %v", id, err) return } + connCtx, connCancel := context.WithCancel(f.ctx) pConn = &udpPacketConn{ conn: inConn, outConn: outConn, lastTime: time.Now(), + cancel: connCancel, } - f.udpForwarder.conns[dstAddr] = pConn + f.udpForwarder.conns[id] = pConn - go f.proxyUDP(pConn, dstAddr) + go f.proxyUDP(connCtx, pConn, id) } - f.udpForwarder.Unlock() } -func (f *Forwarder) proxyUDP(pConn *udpPacketConn, dstAddr string) { +func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack.TransportEndpointID) { defer func() { + pConn.cancel() if err := pConn.conn.Close(); err != nil { - log.Errorf("forwarder: inConn close error: %v", err) + log.Errorf("forwarder: UDP inConn close error for %v: %v", id, err) } if err := pConn.outConn.Close(); err != nil { - log.Errorf("forwarder: outConn close error: %v", err) + log.Errorf("forwarder: UDP outConn close error for %v: %v", id, err) } + + f.udpForwarder.Lock() + delete(f.udpForwarder.conns, id) + f.udpForwarder.Unlock() }() - var wg sync.WaitGroup - wg.Add(2) + errChan := make(chan error, 2) - // Handle outbound to inbound traffic go func() { - defer wg.Done() - f.copyUDP(pConn.conn, pConn.outConn, dstAddr, "outbound->inbound") + errChan <- f.copyUDP(ctx, pConn.conn, pConn.outConn, id, "outbound->inbound") }() - // Handle inbound to outbound traffic go func() { - defer wg.Done() - f.copyUDP(pConn.outConn, pConn.conn, dstAddr, "inbound->outbound") + errChan <- f.copyUDP(ctx, pConn.outConn, pConn.conn, id, "inbound->outbound") }() - wg.Wait() - - // Clean up the connection from the map - f.udpForwarder.Lock() - delete(f.udpForwarder.conns, dstAddr) - f.udpForwarder.Unlock() + select { + case <-ctx.Done(): + return + case err := <-errChan: + if err != nil && !isClosedError(err) { + log.Errorf("forwader: UDP proxy error for %v: %v", id, err) + } + return + } } -func (f *Forwarder) copyUDP(dst net.Conn, src net.Conn, dstAddr, direction string) { - buffer := make([]byte, 65535) +func (f *Forwarder) copyUDP(ctx context.Context, dst net.Conn, src net.Conn, id stack.TransportEndpointID, direction string) error { + bufp := f.udpForwarder.bufPool.Get().(*[]byte) + defer f.udpForwarder.bufPool.Put(bufp) + buffer := *bufp + + if err := src.SetReadDeadline(time.Now().Add(udpTimeout)); err != nil { + return fmt.Errorf("set read deadline: %w", err) + } + if err := src.SetWriteDeadline(time.Now().Add(udpTimeout)); err != nil { + return fmt.Errorf("set write deadline: %w", err) + } + for { - n, err := src.Read(buffer) - if err != nil { - log.Errorf("UDP %s read error: %v", direction, err) - return - } + select { + case <-ctx.Done(): + return ctx.Err() + default: + n, err := src.Read(buffer) + if err != nil { + if isTimeout(err) { + continue + } + return fmt.Errorf("read from %s: %w", direction, err) + } - _, err = dst.Write(buffer[:n]) - if err != nil { - log.Errorf("UDP %s write error: %v", direction, err) - continue - } + _, err = dst.Write(buffer[:n]) + if err != nil { + return fmt.Errorf("write to %s: %w", direction, err) + } - f.udpForwarder.Lock() - if conn, ok := f.udpForwarder.conns[dstAddr]; ok { - conn.lastTime = time.Now() + f.udpForwarder.Lock() + if conn, ok := f.udpForwarder.conns[id]; ok { + conn.lastTime = time.Now() + } + f.udpForwarder.Unlock() } - f.udpForwarder.Unlock() } } + +func isClosedError(err error) bool { + return errors.Is(err, net.ErrClosed) || errors.Is(err, context.Canceled) +} + +func isTimeout(err error) bool { + var netErr net.Error + if errors.As(err, &netErr) { + return netErr.Timeout() + } + return false +} From d2616544fe3d2532bf374d4a20c2d9e585fdd87a Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 30 Dec 2024 15:18:21 +0100 Subject: [PATCH 04/59] Add logger --- client/firewall/uspfilter/allow_netbird.go | 19 +- .../uspfilter/allow_netbird_windows.go | 16 +- .../uspfilter/conntrack/common_test.go | 4 +- client/firewall/uspfilter/conntrack/icmp.go | 10 +- .../firewall/uspfilter/conntrack/icmp_test.go | 4 +- client/firewall/uspfilter/conntrack/tcp.go | 10 +- .../firewall/uspfilter/conntrack/tcp_test.go | 14 +- client/firewall/uspfilter/conntrack/udp.go | 10 +- .../firewall/uspfilter/conntrack/udp_test.go | 10 +- .../firewall/uspfilter/forwarder/endpoint.go | 6 +- .../firewall/uspfilter/forwarder/forwarder.go | 9 +- client/firewall/uspfilter/forwarder/tcp.go | 16 +- client/firewall/uspfilter/forwarder/udp.go | 35 +-- client/firewall/uspfilter/log/log.go | 208 ++++++++++++++++++ client/firewall/uspfilter/log/ringbuffer.go | 93 ++++++++ client/firewall/uspfilter/uspfilter.go | 30 ++- client/firewall/uspfilter/uspfilter_test.go | 4 +- 17 files changed, 436 insertions(+), 62 deletions(-) create mode 100644 client/firewall/uspfilter/log/log.go create mode 100644 client/firewall/uspfilter/log/ringbuffer.go diff --git a/client/firewall/uspfilter/allow_netbird.go b/client/firewall/uspfilter/allow_netbird.go index 29709509011..03f23f5e622 100644 --- a/client/firewall/uspfilter/allow_netbird.go +++ b/client/firewall/uspfilter/allow_netbird.go @@ -3,6 +3,11 @@ package uspfilter import ( + "context" + "time" + + log "github.com/sirupsen/logrus" + "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" "github.com/netbirdio/netbird/client/internal/statemanager" ) @@ -17,23 +22,31 @@ func (m *Manager) Reset(stateManager *statemanager.Manager) error { if m.udpTracker != nil { m.udpTracker.Close() - m.udpTracker = conntrack.NewUDPTracker(conntrack.DefaultUDPTimeout) + m.udpTracker = conntrack.NewUDPTracker(conntrack.DefaultUDPTimeout, m.logger) } if m.icmpTracker != nil { m.icmpTracker.Close() - m.icmpTracker = conntrack.NewICMPTracker(conntrack.DefaultICMPTimeout) + m.icmpTracker = conntrack.NewICMPTracker(conntrack.DefaultICMPTimeout, m.logger) } if m.tcpTracker != nil { m.tcpTracker.Close() - m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout) + m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout, m.logger) } if m.forwarder != nil { m.forwarder.Stop() } + if m.logger != nil { + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + if err := m.logger.Stop(ctx); err != nil { + log.Errorf("failed to shutdown logger: %v", err) + } + } + if m.nativeFirewall != nil { return m.nativeFirewall.Reset(stateManager) } diff --git a/client/firewall/uspfilter/allow_netbird_windows.go b/client/firewall/uspfilter/allow_netbird_windows.go index 42bf0896ec4..37958597826 100644 --- a/client/firewall/uspfilter/allow_netbird_windows.go +++ b/client/firewall/uspfilter/allow_netbird_windows.go @@ -1,9 +1,11 @@ package uspfilter import ( + "context" "fmt" "os/exec" "syscall" + "time" log "github.com/sirupsen/logrus" @@ -29,23 +31,31 @@ func (m *Manager) Reset(*statemanager.Manager) error { if m.udpTracker != nil { m.udpTracker.Close() - m.udpTracker = conntrack.NewUDPTracker(conntrack.DefaultUDPTimeout) + m.udpTracker = conntrack.NewUDPTracker(conntrack.DefaultUDPTimeout, m.logger) } if m.icmpTracker != nil { m.icmpTracker.Close() - m.icmpTracker = conntrack.NewICMPTracker(conntrack.DefaultICMPTimeout) + m.icmpTracker = conntrack.NewICMPTracker(conntrack.DefaultICMPTimeout, m.logger) } if m.tcpTracker != nil { m.tcpTracker.Close() - m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout) + m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout, m.logger) } if m.forwarder != nil { m.forwarder.Stop() } + if m.logger != nil { + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + if err := m.logger.Stop(ctx); err != nil { + log.Errorf("failed to shutdown logger: %v", err) + } + } + if !isWindowsFirewallReachable() { return nil } diff --git a/client/firewall/uspfilter/conntrack/common_test.go b/client/firewall/uspfilter/conntrack/common_test.go index 72d006def57..b885470a35d 100644 --- a/client/firewall/uspfilter/conntrack/common_test.go +++ b/client/firewall/uspfilter/conntrack/common_test.go @@ -64,7 +64,7 @@ func BenchmarkAtomicOperations(b *testing.B) { // Memory pressure tests func BenchmarkMemoryPressure(b *testing.B) { b.Run("TCPHighLoad", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout) + tracker := NewTCPTracker(DefaultTCPTimeout, nil) defer tracker.Close() // Generate different IPs @@ -89,7 +89,7 @@ func BenchmarkMemoryPressure(b *testing.B) { }) b.Run("UDPHighLoad", func(b *testing.B) { - tracker := NewUDPTracker(DefaultUDPTimeout) + tracker := NewUDPTracker(DefaultUDPTimeout, nil) defer tracker.Close() // Generate different IPs diff --git a/client/firewall/uspfilter/conntrack/icmp.go b/client/firewall/uspfilter/conntrack/icmp.go index e0a971678f1..277a4b26ed4 100644 --- a/client/firewall/uspfilter/conntrack/icmp.go +++ b/client/firewall/uspfilter/conntrack/icmp.go @@ -6,6 +6,8 @@ import ( "time" "github.com/google/gopacket/layers" + + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) const ( @@ -33,6 +35,7 @@ type ICMPConnTrack struct { // ICMPTracker manages ICMP connection states type ICMPTracker struct { + logger *nblog.Logger connections map[ICMPConnKey]*ICMPConnTrack timeout time.Duration cleanupTicker *time.Ticker @@ -42,12 +45,13 @@ type ICMPTracker struct { } // NewICMPTracker creates a new ICMP connection tracker -func NewICMPTracker(timeout time.Duration) *ICMPTracker { +func NewICMPTracker(timeout time.Duration, logger *nblog.Logger) *ICMPTracker { if timeout == 0 { timeout = DefaultICMPTimeout } tracker := &ICMPTracker{ + logger: logger, connections: make(map[ICMPConnKey]*ICMPConnTrack), timeout: timeout, cleanupTicker: time.NewTicker(ICMPCleanupInterval), @@ -83,6 +87,8 @@ func (t *ICMPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, id uint16, seq u conn.lastSeen.Store(now) conn.established.Store(true) t.connections[key] = conn + + t.logger.Trace("New ICMP connection %v", key) } t.mutex.Unlock() @@ -141,6 +147,8 @@ func (t *ICMPTracker) cleanup() { t.ipPool.Put(conn.SourceIP) t.ipPool.Put(conn.DestIP) delete(t.connections, key) + + t.logger.Debug("ICMPTracker: removed connection %v", key) } } } diff --git a/client/firewall/uspfilter/conntrack/icmp_test.go b/client/firewall/uspfilter/conntrack/icmp_test.go index 21176e719d4..e653416f98f 100644 --- a/client/firewall/uspfilter/conntrack/icmp_test.go +++ b/client/firewall/uspfilter/conntrack/icmp_test.go @@ -7,7 +7,7 @@ import ( func BenchmarkICMPTracker(b *testing.B) { b.Run("TrackOutbound", func(b *testing.B) { - tracker := NewICMPTracker(DefaultICMPTimeout) + tracker := NewICMPTracker(DefaultICMPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -20,7 +20,7 @@ func BenchmarkICMPTracker(b *testing.B) { }) b.Run("IsValidInbound", func(b *testing.B) { - tracker := NewICMPTracker(DefaultICMPTimeout) + tracker := NewICMPTracker(DefaultICMPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") diff --git a/client/firewall/uspfilter/conntrack/tcp.go b/client/firewall/uspfilter/conntrack/tcp.go index a7968dc7375..a42208b61fe 100644 --- a/client/firewall/uspfilter/conntrack/tcp.go +++ b/client/firewall/uspfilter/conntrack/tcp.go @@ -6,6 +6,8 @@ import ( "net" "sync" "time" + + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) const ( @@ -67,6 +69,7 @@ type TCPConnTrack struct { // TCPTracker manages TCP connection states type TCPTracker struct { + logger *nblog.Logger connections map[ConnKey]*TCPConnTrack mutex sync.RWMutex cleanupTicker *time.Ticker @@ -76,8 +79,9 @@ type TCPTracker struct { } // NewTCPTracker creates a new TCP connection tracker -func NewTCPTracker(timeout time.Duration) *TCPTracker { +func NewTCPTracker(timeout time.Duration, logger *nblog.Logger) *TCPTracker { tracker := &TCPTracker{ + logger: logger, connections: make(map[ConnKey]*TCPConnTrack), cleanupTicker: time.NewTicker(TCPCleanupInterval), done: make(chan struct{}), @@ -116,6 +120,8 @@ func (t *TCPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, d conn.lastSeen.Store(now) conn.established.Store(false) t.connections[key] = conn + + t.logger.Trace("New TCP connection: %s:%d -> %s:%d", srcIP, srcPort, dstIP, dstPort) } t.mutex.Unlock() @@ -318,6 +324,8 @@ func (t *TCPTracker) cleanup() { t.ipPool.Put(conn.SourceIP) t.ipPool.Put(conn.DestIP) delete(t.connections, key) + + t.logger.Trace("Closed TCP connection: %s:%d -> %s:%d", conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) } } } diff --git a/client/firewall/uspfilter/conntrack/tcp_test.go b/client/firewall/uspfilter/conntrack/tcp_test.go index 6c8f82423bd..c44e7dfa735 100644 --- a/client/firewall/uspfilter/conntrack/tcp_test.go +++ b/client/firewall/uspfilter/conntrack/tcp_test.go @@ -9,7 +9,7 @@ import ( ) func TestTCPStateMachine(t *testing.T) { - tracker := NewTCPTracker(DefaultTCPTimeout) + tracker := NewTCPTracker(DefaultTCPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("100.64.0.1") @@ -154,7 +154,7 @@ func TestTCPStateMachine(t *testing.T) { t.Run(tt.name, func(t *testing.T) { t.Helper() - tracker = NewTCPTracker(DefaultTCPTimeout) + tracker = NewTCPTracker(DefaultTCPTimeout, nil) tt.test(t) }) } @@ -162,7 +162,7 @@ func TestTCPStateMachine(t *testing.T) { } func TestRSTHandling(t *testing.T) { - tracker := NewTCPTracker(DefaultTCPTimeout) + tracker := NewTCPTracker(DefaultTCPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("100.64.0.1") @@ -233,7 +233,7 @@ func establishConnection(t *testing.T, tracker *TCPTracker, srcIP, dstIP net.IP, func BenchmarkTCPTracker(b *testing.B) { b.Run("TrackOutbound", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout) + tracker := NewTCPTracker(DefaultTCPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -246,7 +246,7 @@ func BenchmarkTCPTracker(b *testing.B) { }) b.Run("IsValidInbound", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout) + tracker := NewTCPTracker(DefaultTCPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -264,7 +264,7 @@ func BenchmarkTCPTracker(b *testing.B) { }) b.Run("ConcurrentAccess", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout) + tracker := NewTCPTracker(DefaultTCPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -287,7 +287,7 @@ func BenchmarkTCPTracker(b *testing.B) { // Benchmark connection cleanup func BenchmarkCleanup(b *testing.B) { b.Run("TCPCleanup", func(b *testing.B) { - tracker := NewTCPTracker(100 * time.Millisecond) // Short timeout for testing + tracker := NewTCPTracker(100*time.Millisecond, nil) // Short timeout for testing defer tracker.Close() // Pre-populate with expired connections diff --git a/client/firewall/uspfilter/conntrack/udp.go b/client/firewall/uspfilter/conntrack/udp.go index a969a4e8425..63000634968 100644 --- a/client/firewall/uspfilter/conntrack/udp.go +++ b/client/firewall/uspfilter/conntrack/udp.go @@ -4,6 +4,8 @@ import ( "net" "sync" "time" + + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) const ( @@ -20,6 +22,7 @@ type UDPConnTrack struct { // UDPTracker manages UDP connection states type UDPTracker struct { + logger *nblog.Logger connections map[ConnKey]*UDPConnTrack timeout time.Duration cleanupTicker *time.Ticker @@ -29,12 +32,13 @@ type UDPTracker struct { } // NewUDPTracker creates a new UDP connection tracker -func NewUDPTracker(timeout time.Duration) *UDPTracker { +func NewUDPTracker(timeout time.Duration, logger *nblog.Logger) *UDPTracker { if timeout == 0 { timeout = DefaultUDPTimeout } tracker := &UDPTracker{ + logger: logger, connections: make(map[ConnKey]*UDPConnTrack), timeout: timeout, cleanupTicker: time.NewTicker(UDPCleanupInterval), @@ -70,6 +74,8 @@ func (t *UDPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, d conn.lastSeen.Store(now) conn.established.Store(true) t.connections[key] = conn + + t.logger.Trace("New UDP connection: %s", conn) } t.mutex.Unlock() @@ -120,6 +126,8 @@ func (t *UDPTracker) cleanup() { t.ipPool.Put(conn.SourceIP) t.ipPool.Put(conn.DestIP) delete(t.connections, key) + + t.logger.Trace("UDP connection timed out: %s", conn) } } } diff --git a/client/firewall/uspfilter/conntrack/udp_test.go b/client/firewall/uspfilter/conntrack/udp_test.go index 67172189069..4e42c484f47 100644 --- a/client/firewall/uspfilter/conntrack/udp_test.go +++ b/client/firewall/uspfilter/conntrack/udp_test.go @@ -29,7 +29,7 @@ func TestNewUDPTracker(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tracker := NewUDPTracker(tt.timeout) + tracker := NewUDPTracker(tt.timeout, nil) assert.NotNil(t, tracker) assert.Equal(t, tt.wantTimeout, tracker.timeout) assert.NotNil(t, tracker.connections) @@ -40,7 +40,7 @@ func TestNewUDPTracker(t *testing.T) { } func TestUDPTracker_TrackOutbound(t *testing.T) { - tracker := NewUDPTracker(DefaultUDPTimeout) + tracker := NewUDPTracker(DefaultUDPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.2") @@ -63,7 +63,7 @@ func TestUDPTracker_TrackOutbound(t *testing.T) { } func TestUDPTracker_IsValidInbound(t *testing.T) { - tracker := NewUDPTracker(1 * time.Second) + tracker := NewUDPTracker(1*time.Second, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.2") @@ -211,7 +211,7 @@ func TestUDPTracker_Cleanup(t *testing.T) { func BenchmarkUDPTracker(b *testing.B) { b.Run("TrackOutbound", func(b *testing.B) { - tracker := NewUDPTracker(DefaultUDPTimeout) + tracker := NewUDPTracker(DefaultUDPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -224,7 +224,7 @@ func BenchmarkUDPTracker(b *testing.B) { }) b.Run("IsValidInbound", func(b *testing.B) { - tracker := NewUDPTracker(DefaultUDPTimeout) + tracker := NewUDPTracker(DefaultUDPTimeout, nil) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") diff --git a/client/firewall/uspfilter/forwarder/endpoint.go b/client/firewall/uspfilter/forwarder/endpoint.go index 9f22fe3a20d..c234ca24136 100644 --- a/client/firewall/uspfilter/forwarder/endpoint.go +++ b/client/firewall/uspfilter/forwarder/endpoint.go @@ -1,15 +1,17 @@ package forwarder import ( - log "github.com/sirupsen/logrus" wgdevice "golang.zx2c4.com/wireguard/device" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/header" "gvisor.dev/gvisor/pkg/tcpip/stack" + + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) // endpoint implements stack.LinkEndpoint and handles integration with the wireguard device type endpoint struct { + logger *nblog.Logger dispatcher stack.NetworkDispatcher device *wgdevice.Device mtu uint32 @@ -55,7 +57,7 @@ func (e *endpoint) WritePackets(pkts stack.PacketBufferList) (int, tcpip.Error) // TODO: handle dest ip addresses outside our network err := e.device.CreateOutboundPacket(data.AsSlice(), address.AsSlice()) if err != nil { - log.Errorf("CreateOutboundPacket: %v", err) + e.logger.Error("CreateOutboundPacket: %v", err) continue } written++ diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index 815c7da09c6..f3920065851 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -14,6 +14,7 @@ import ( "gvisor.dev/gvisor/pkg/tcpip/transport/udp" "github.com/netbirdio/netbird/client/firewall/uspfilter/common" + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) const ( @@ -22,6 +23,7 @@ const ( ) type Forwarder struct { + logger *nblog.Logger stack *stack.Stack endpoint *endpoint udpForwarder *udpForwarder @@ -29,8 +31,7 @@ type Forwarder struct { cancel context.CancelFunc } -func New(iface common.IFaceMapper) (*Forwarder, error) { - +func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { s := stack.New(stack.Options{ NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, TransportProtocols: []stack.TransportProtocolFactory{ @@ -46,6 +47,7 @@ func New(iface common.IFaceMapper) (*Forwarder, error) { } nicID := tcpip.NICID(1) endpoint := &endpoint{ + logger: logger, device: iface.GetWGDevice(), mtu: uint32(mtu), } @@ -91,9 +93,10 @@ func New(iface common.IFaceMapper) (*Forwarder, error) { ctx, cancel := context.WithCancel(context.Background()) f := &Forwarder{ + logger: logger, stack: s, endpoint: endpoint, - udpForwarder: newUDPForwarder(), + udpForwarder: newUDPForwarder(logger), ctx: ctx, cancel: cancel, } diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index 90967b6f57f..25503cb6d47 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -6,7 +6,6 @@ import ( "io" "net" - log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" "gvisor.dev/gvisor/pkg/waiter" @@ -23,16 +22,19 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { outConn, err := (&net.Dialer{}).DialContext(f.ctx, "tcp", dialAddr) if err != nil { r.Complete(true) + f.logger.Trace("forwarder: dial error for %v: %v", id, err) return } + f.logger.Trace("forwarder: established TCP connection to %v", id) + // Create wait queue for blocking syscalls wq := waiter.Queue{} ep, err2 := r.CreateEndpoint(&wq) if err2 != nil { if err := outConn.Close(); err != nil { - log.Errorf("forwarder: outConn close error: %v", err) + f.logger.Error("forwarder: outConn close error: %v", err) } r.Complete(true) return @@ -49,10 +51,10 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { defer func() { if err := inConn.Close(); err != nil { - log.Errorf("forwarder: inConn close error: %v", err) + f.logger.Error("forwarder: inConn close error: %v", err) } if err := outConn.Close(); err != nil { - log.Errorf("forwarder: outConn close error: %v", err) + f.logger.Error("forwarder: outConn close error: %v", err) } }() @@ -65,7 +67,7 @@ func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { go func() { n, err := io.Copy(outConn, inConn) if err != nil && !isClosedError(err) { - log.Errorf("proxyTCP: inbound->outbound copy error after %d bytes: %v", n, err) + f.logger.Error("inbound->outbound copy error after %d bytes: %v", n, err) } errChan <- err }() @@ -73,7 +75,7 @@ func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { go func() { n, err := io.Copy(inConn, outConn) if err != nil && !isClosedError(err) { - log.Errorf("proxyTCP: outbound->inbound copy error after %d bytes: %v", n, err) + f.logger.Error("outbound->inbound copy error after %d bytes: %v", n, err) } errChan <- err }() @@ -83,7 +85,7 @@ func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { return case err := <-errChan: if err != nil && !isClosedError(err) { - log.Errorf("proxyTCP: copy error: %v", err) + f.logger.Error("proxyTCP: copy error: %v", err) } return } diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 7d201024eb0..bb43a8346e3 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -8,11 +8,12 @@ import ( "sync" "time" - log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" "gvisor.dev/gvisor/pkg/waiter" + + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) const ( @@ -29,15 +30,17 @@ type udpPacketConn struct { type udpForwarder struct { sync.RWMutex + logger *nblog.Logger conns map[stack.TransportEndpointID]*udpPacketConn bufPool sync.Pool ctx context.Context cancel context.CancelFunc } -func newUDPForwarder() *udpForwarder { +func newUDPForwarder(logger *nblog.Logger) *udpForwarder { ctx, cancel := context.WithCancel(context.Background()) f := &udpForwarder{ + logger: logger, conns: make(map[stack.TransportEndpointID]*udpPacketConn), ctx: ctx, cancel: cancel, @@ -62,10 +65,10 @@ func (f *udpForwarder) Stop() { for id, conn := range f.conns { conn.cancel() if err := conn.conn.Close(); err != nil { - log.Errorf("forwarder: UDP conn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP conn close error for %v: %v", id, err) } if err := conn.outConn.Close(); err != nil { - log.Errorf("forwarder: UDP outConn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) } delete(f.conns, id) } @@ -87,13 +90,13 @@ func (f *udpForwarder) cleanup() { if now.Sub(conn.lastTime) > udpTimeout { conn.cancel() if err := conn.conn.Close(); err != nil { - log.Errorf("forwarder: UDP conn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP conn close error for %v: %v", id, err) } if err := conn.outConn.Close(); err != nil { - log.Errorf("forwarder: UDP outConn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) } delete(f.conns, id) - log.Debugf("forwarder: cleaned up idle UDP connection %v", id) + f.logger.Trace("forwarder: cleaned up idle UDP connection %v", id) } } f.Unlock() @@ -107,7 +110,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { dstAddr := fmt.Sprintf("%s:%d", id.LocalAddress.String(), id.LocalPort) if f.ctx.Err() != nil { - log.Debug("forwarder: context done, dropping UDP packet") + f.logger.Trace("forwarder: context done, dropping UDP packet") return } @@ -116,7 +119,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { ep, err := r.CreateEndpoint(&wq) if err != nil { - log.Errorf("forwarder: failed to create UDP endpoint: %v", err) + f.logger.Error("forwarder: failed to create UDP endpoint: %v", err) return } @@ -131,12 +134,16 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { outConn, err := (&net.Dialer{}).DialContext(f.ctx, "udp", dstAddr) if err != nil { if err := inConn.Close(); err != nil { - log.Errorf("forwarder: UDP inConn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP inConn close error for %v: %v", id, err) } - log.Errorf("forwarder: UDP dial error for %v: %v", id, err) + f.logger.Debug("forwarder: UDP dial error for %v: %v", id, err) + + // TODO: Send ICMP error message return } + f.logger.Trace("forwarder: established UDP connection to %v", id) + connCtx, connCancel := context.WithCancel(f.ctx) pConn = &udpPacketConn{ conn: inConn, @@ -154,10 +161,10 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack defer func() { pConn.cancel() if err := pConn.conn.Close(); err != nil { - log.Errorf("forwarder: UDP inConn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP inConn close error for %v: %v", id, err) } if err := pConn.outConn.Close(); err != nil { - log.Errorf("forwarder: UDP outConn close error for %v: %v", id, err) + f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) } f.udpForwarder.Lock() @@ -180,7 +187,7 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack return case err := <-errChan: if err != nil && !isClosedError(err) { - log.Errorf("forwader: UDP proxy error for %v: %v", id, err) + f.logger.Error("proxyUDP: copy error: %v", err) } return } diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go new file mode 100644 index 00000000000..2e9a4d4b729 --- /dev/null +++ b/client/firewall/uspfilter/log/log.go @@ -0,0 +1,208 @@ +// Package logger provides a high-performance, non-blocking logger for userspace networking +package log + +import ( + "context" + "fmt" + "io" + "sync" + "sync/atomic" + "time" + + log "github.com/sirupsen/logrus" +) + +const ( + maxBatchSize = 1024 * 16 // 16KB max batch size + maxMessageSize = 1024 * 2 // 2KB per message + bufferSize = 1024 * 256 // 256KB ring buffer + defaultFlushInterval = 2 * time.Second +) + +// Level represents log severity +type Level uint32 + +const ( + LevelPanic Level = iota + LevelFatal + LevelError + LevelWarn + LevelInfo + LevelDebug + LevelTrace +) + +var levelStrings = map[Level]string{ + LevelPanic: "PANC", + LevelFatal: "FATL", + LevelError: "ERRO", + LevelWarn: "WARN", + LevelInfo: "INFO", + LevelDebug: "DEBG", + LevelTrace: "TRAC", +} + +func FromLogrusLevel(level log.Level) Level { + switch level { + case log.TraceLevel: + return LevelTrace + case log.DebugLevel: + return LevelDebug + case log.InfoLevel: + return LevelInfo + case log.WarnLevel: + return LevelWarn + case log.ErrorLevel: + return LevelError + case log.FatalLevel: + return LevelFatal + case log.PanicLevel: + return LevelPanic + default: + return LevelInfo + } +} + +// Logger is a high-performance, non-blocking logger +type Logger struct { + output io.Writer + level atomic.Uint32 + buffer *ringBuffer + shutdown chan struct{} + wg sync.WaitGroup + + // Reusable buffer pool for formatting messages + bufPool sync.Pool +} + +func NewFromLogrus(logrusLogger *log.Logger) *Logger { + l := &Logger{ + output: logrusLogger.Out, + buffer: newRingBuffer(bufferSize), + shutdown: make(chan struct{}), + bufPool: sync.Pool{ + New: func() interface{} { + // Pre-allocate buffer for message formatting + b := make([]byte, 0, maxMessageSize) + return &b + }, + }, + } + l.level.Store(uint32(LevelInfo)) + + l.wg.Add(1) + go l.worker() + + return l +} + +func (l *Logger) SetLevel(level Level) { + l.level.Store(uint32(level)) +} + +func (l *Logger) formatMessage(buf *[]byte, level Level, format string, args ...interface{}) { + *buf = (*buf)[:0] + + // Timestamp + *buf = time.Now().AppendFormat(*buf, "2006-01-02T15:04:05.000000-07:00") + *buf = append(*buf, ' ') + + // Level + *buf = append(*buf, levelStrings[level]...) + *buf = append(*buf, ' ') + + // Message + if len(args) > 0 { + *buf = append(*buf, fmt.Sprintf(format, args...)...) + } else { + *buf = append(*buf, format...) + } + + *buf = append(*buf, '\n') +} + +func (l *Logger) log(level Level, format string, args ...interface{}) { + bufp := l.bufPool.Get().(*[]byte) + l.formatMessage(bufp, level, format, args...) + + if len(*bufp) > maxMessageSize { + *bufp = (*bufp)[:maxMessageSize] + } + l.buffer.Write(*bufp) + + l.bufPool.Put(bufp) +} + +func (l *Logger) Trace(format string, args ...interface{}) { + if l.level.Load() <= uint32(LevelTrace) { + l.log(LevelTrace, format, args...) + } +} + +func (l *Logger) Debug(format string, args ...interface{}) { + if l.level.Load() <= uint32(LevelDebug) { + l.log(LevelDebug, format, args...) + } +} + +func (l *Logger) Info(format string, args ...interface{}) { + if l.level.Load() <= uint32(LevelInfo) { + l.log(LevelInfo, format, args...) + } +} + +func (l *Logger) Warn(format string, args ...interface{}) { + if l.level.Load() <= uint32(LevelWarn) { + l.log(LevelWarn, format, args...) + } +} + +func (l *Logger) Error(format string, args ...interface{}) { + if l.level.Load() <= uint32(LevelError) { + l.log(LevelError, format, args...) + } +} + +// worker periodically flushes the buffer +func (l *Logger) worker() { + defer l.wg.Done() + + ticker := time.NewTicker(defaultFlushInterval) + defer ticker.Stop() + + buf := make([]byte, 0, maxBatchSize) + + for { + select { + case <-l.shutdown: + return + case <-ticker.C: + // Read accumulated messages + n, _ := l.buffer.Read(buf[:cap(buf)]) + if n == 0 { + continue + } + + // Write batch + l.output.Write(buf[:n]) + } + } +} + +// Stop gracefully shuts down the logger +func (l *Logger) Stop(ctx context.Context) error { + close(l.shutdown) + + done := make(chan struct{}) + go func() { + l.wg.Wait() + close(done) + }() + + select { + case <-ctx.Done(): + return ctx.Err() + case <-done: + return nil + } +} diff --git a/client/firewall/uspfilter/log/ringbuffer.go b/client/firewall/uspfilter/log/ringbuffer.go new file mode 100644 index 00000000000..48ebe84aee4 --- /dev/null +++ b/client/firewall/uspfilter/log/ringbuffer.go @@ -0,0 +1,93 @@ +package log + +import "sync" + +// ringBuffer is a simple ring buffer implementation +type ringBuffer struct { + buf []byte + size int + r, w int64 // Read and write positions + mu sync.Mutex +} + +func newRingBuffer(size int) *ringBuffer { + return &ringBuffer{ + buf: make([]byte, size), + size: size, + } +} + +func (r *ringBuffer) Write(p []byte) (n int, err error) { + if len(p) == 0 { + return 0, nil + } + + r.mu.Lock() + defer r.mu.Unlock() + + if len(p) > r.size { + p = p[:r.size] + } + + n = len(p) + + // Write data, handling wrap-around + pos := int(r.w % int64(r.size)) + writeLen := min(len(p), r.size-pos) + copy(r.buf[pos:], p[:writeLen]) + + // If we have more data and need to wrap around + if writeLen < len(p) { + copy(r.buf, p[writeLen:]) + } + + // Update write position + r.w += int64(n) + + return n, nil +} + +func (r *ringBuffer) Read(p []byte) (n int, err error) { + r.mu.Lock() + defer r.mu.Unlock() + + if r.w == r.r { + return 0, nil + } + + // Calculate available data accounting for wraparound + available := int(r.w - r.r) + if available < 0 { + available += r.size + } + available = min(available, r.size) + + // Limit read to buffer size + toRead := min(available, len(p)) + if toRead == 0 { + return 0, nil + } + + // Read data, handling wrap-around + pos := int(r.r % int64(r.size)) + readLen := min(toRead, r.size-pos) + n = copy(p, r.buf[pos:pos+readLen]) + + // If we need more data and need to wrap around + if readLen < toRead { + n += copy(p[readLen:toRead], r.buf[:toRead-readLen]) + } + + // Update read position + r.r += int64(n) + + return n, nil +} + +// min returns the smaller of two integers +func min(a, b int) int { + if a < b { + return a + } + return b +} diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index feed1887b44..55e2063ecba 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -17,6 +17,7 @@ import ( "github.com/netbirdio/netbird/client/firewall/uspfilter/common" "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" "github.com/netbirdio/netbird/client/firewall/uspfilter/forwarder" + nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" "github.com/netbirdio/netbird/client/internal/statemanager" ) @@ -52,6 +53,7 @@ type Manager struct { icmpTracker *conntrack.ICMPTracker tcpTracker *conntrack.TCPTracker forwarder *forwarder.Forwarder + logger *nblog.Logger } // decoder for packages @@ -106,15 +108,17 @@ func create(iface common.IFaceMapper) (*Manager, error) { stateful: !disableConntrack, // TODO: fix routingEnabled: true, + // TODO: support chaning log level from logrus + logger: nblog.NewFromLogrus(log.StandardLogger()), } // Only initialize trackers if stateful mode is enabled if disableConntrack { log.Info("conntrack is disabled") } else { - m.udpTracker = conntrack.NewUDPTracker(conntrack.DefaultUDPTimeout) - m.icmpTracker = conntrack.NewICMPTracker(conntrack.DefaultICMPTimeout) - m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout) + m.udpTracker = conntrack.NewUDPTracker(conntrack.DefaultUDPTimeout, m.logger) + m.icmpTracker = conntrack.NewICMPTracker(conntrack.DefaultICMPTimeout, m.logger) + m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout, m.logger) } intf := iface.GetWGDevice() @@ -125,7 +129,7 @@ func create(iface common.IFaceMapper) (*Manager, error) { m.routingEnabled = false } else { var err error - m.forwarder, err = forwarder.New(iface) + m.forwarder, err = forwarder.New(iface, m.logger) if err != nil { log.Errorf("failed to create forwarder: %v", err) m.routingEnabled = false @@ -455,17 +459,16 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { defer m.decoders.Put(d) if !m.isValidPacket(d, packetData) { - log.Debugf("invalid packet: %v", d.decoded) + m.logger.Trace("Invalid packet structure") return true } srcIP, dstIP := m.extractIPs(d) if srcIP == nil { - log.Errorf("unknown layer: %v", d.decoded[0]) + m.logger.Error("Unknown network layer: %v", d.decoded[0]) return true } - // Check if this is local or routed traffic isLocal := m.isLocalIP(dstIP) // For all inbound traffic, first check if it matches a tracked connection. @@ -476,7 +479,12 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { // Handle local traffic - apply peer ACLs if isLocal { - return m.applyRules(srcIP, packetData, rules, d) + drop := m.applyRules(srcIP, packetData, rules, d) + if drop { + m.logger.Trace("Dropping local packet: src=%s dst=%s rules=denied", + srcIP, dstIP) + } + return drop } // Handle routed traffic @@ -484,6 +492,8 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { // We might need to apply NAT // Don't handle routing if not enabled if !m.routingEnabled { + m.logger.Trace("Dropping routed packet (routing disabled): src=%s dst=%s", + srcIP, dstIP) return true } @@ -493,13 +503,15 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { // Check route ACLs if !m.checkRouteACLs(srcIP, dstIP, proto, srcPort, dstPort) { + m.logger.Trace("Dropping routed packet (ACL denied): src=%s:%d dst=%s:%d proto=%v", + srcIP, srcPort, dstIP, dstPort, proto) return true } // Let forwarder handle the packet if it passed route ACLs err := m.forwarder.InjectIncomingPacket(packetData) if err != nil { - log.Errorf("Failed to inject incoming packet: %v", err) + m.logger.Error("Failed to inject incoming packet: %v", err) } // Default: drop diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index 443d82607ea..2d85116d452 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -400,7 +400,7 @@ func TestProcessOutgoingHooks(t *testing.T) { Mask: net.CIDRMask(16, 32), } manager.udpTracker.Close() - manager.udpTracker = conntrack.NewUDPTracker(100 * time.Millisecond) + manager.udpTracker = conntrack.NewUDPTracker(100*time.Millisecond, nil) defer func() { require.NoError(t, manager.Reset(nil)) }() @@ -518,7 +518,7 @@ func TestStatefulFirewall_UDPTracking(t *testing.T) { } manager.udpTracker.Close() // Close the existing tracker - manager.udpTracker = conntrack.NewUDPTracker(200 * time.Millisecond) + manager.udpTracker = conntrack.NewUDPTracker(200*time.Millisecond, nil) manager.decoders = sync.Pool{ New: func() any { d := &decoder{ From 6a97d44d5da1dd3a0219f69bf7fcaa7a050cbd4c Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 30 Dec 2024 20:50:20 +0100 Subject: [PATCH 05/59] Improve udp implementation --- client/firewall/uspfilter/forwarder/tcp.go | 8 +- client/firewall/uspfilter/forwarder/udp.go | 143 +++++++++++++-------- 2 files changed, 93 insertions(+), 58 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index 25503cb6d47..263fffb513b 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -26,13 +26,11 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { return } - f.logger.Trace("forwarder: established TCP connection to %v", id) - // Create wait queue for blocking syscalls wq := waiter.Queue{} - ep, err2 := r.CreateEndpoint(&wq) - if err2 != nil { + ep, epErr := r.CreateEndpoint(&wq) + if epErr != nil { if err := outConn.Close(); err != nil { f.logger.Error("forwarder: outConn close error: %v", err) } @@ -45,6 +43,8 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { inConn := gonet.NewTCPConn(&wq, ep) + f.logger.Trace("forwarder: established TCP connection to %v", id) + go f.proxyTCP(inConn, outConn) } diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index bb43a8346e3..a6f3ab993dc 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -6,8 +6,10 @@ import ( "fmt" "net" "sync" + "sync/atomic" "time" + log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" @@ -24,7 +26,7 @@ const ( type udpPacketConn struct { conn *gonet.UDPConn outConn net.Conn - lastTime time.Time + lastSeen atomic.Int64 cancel context.CancelFunc } @@ -84,22 +86,37 @@ func (f *udpForwarder) cleanup() { case <-f.ctx.Done(): return case <-ticker.C: - f.Lock() - now := time.Now() + var idleConns []struct { + id stack.TransportEndpointID + conn *udpPacketConn + } + + f.RLock() for id, conn := range f.conns { - if now.Sub(conn.lastTime) > udpTimeout { - conn.cancel() - if err := conn.conn.Close(); err != nil { - f.logger.Error("forwarder: UDP conn close error for %v: %v", id, err) - } - if err := conn.outConn.Close(); err != nil { - f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) - } - delete(f.conns, id) - f.logger.Trace("forwarder: cleaned up idle UDP connection %v", id) + if conn.getIdleDuration() > udpTimeout { + idleConns = append(idleConns, struct { + id stack.TransportEndpointID + conn *udpPacketConn + }{id, conn}) + } + } + f.RUnlock() + + for _, idle := range idleConns { + idle.conn.cancel() + if err := idle.conn.conn.Close(); err != nil { + f.logger.Error("forwarder: UDP conn close error for %v: %v", idle.id, err) } + if err := idle.conn.outConn.Close(); err != nil { + f.logger.Error("forwarder: UDP outConn close error for %v: %v", idle.id, err) + } + + f.Lock() + delete(f.conns, idle.id) + f.Unlock() + + f.logger.Trace("forwarder: cleaned up idle UDP connection %v", idle.id) } - f.Unlock() } } } @@ -114,47 +131,60 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { return } - // Create wait queue for blocking syscalls - wq := waiter.Queue{} + f.udpForwarder.RLock() + pConn, exists := f.udpForwarder.conns[id] + f.udpForwarder.RUnlock() + if exists { + f.logger.Trace("forwarder: existing UDP connection for %v", id) + return + } - ep, err := r.CreateEndpoint(&wq) + outConn, err := (&net.Dialer{}).DialContext(f.ctx, "udp", dstAddr) if err != nil { - f.logger.Error("forwarder: failed to create UDP endpoint: %v", err) + f.logger.Debug("forwarder: UDP dial error for %v: %v", id, err) + // TODO: Send ICMP error message return } - inConn := gonet.NewUDPConn(f.stack, &wq, ep) + // Create wait queue for blocking syscalls + wq := waiter.Queue{} + ep, epErr := r.CreateEndpoint(&wq) + if epErr != nil { + f.logger.Error("forwarder: failed to create UDP endpoint: %v", epErr) + if err := outConn.Close(); err != nil { + f.logger.Debug("forwarder: UDP outConn close error for %v: %v", id, err) + } + return + } - // Try to get existing connection or create a new one - f.udpForwarder.Lock() - defer f.udpForwarder.Unlock() + inConn := gonet.NewUDPConn(f.stack, &wq, ep) + connCtx, connCancel := context.WithCancel(f.ctx) - pConn, exists := f.udpForwarder.conns[id] - if !exists { - outConn, err := (&net.Dialer{}).DialContext(f.ctx, "udp", dstAddr) - if err != nil { - if err := inConn.Close(); err != nil { - f.logger.Error("forwarder: UDP inConn close error for %v: %v", id, err) - } - f.logger.Debug("forwarder: UDP dial error for %v: %v", id, err) + pConn = &udpPacketConn{ + conn: inConn, + outConn: outConn, + cancel: connCancel, + } + pConn.updateLastSeen() - // TODO: Send ICMP error message - return + f.udpForwarder.Lock() + // Double-check no connection was created while we were setting up + if _, exists := f.udpForwarder.conns[id]; exists { + f.udpForwarder.Unlock() + pConn.cancel() + if err := inConn.Close(); err != nil { + f.logger.Debug("forwarder: UDP inConn close error for %v: %v", id, err) } - - f.logger.Trace("forwarder: established UDP connection to %v", id) - - connCtx, connCancel := context.WithCancel(f.ctx) - pConn = &udpPacketConn{ - conn: inConn, - outConn: outConn, - lastTime: time.Now(), - cancel: connCancel, + if err := outConn.Close(); err != nil { + f.logger.Debug("forwarder: UDP outConn close error for %v: %v", id, err) } - f.udpForwarder.conns[id] = pConn - - go f.proxyUDP(connCtx, pConn, id) + return } + f.udpForwarder.conns[id] = pConn + f.udpForwarder.Unlock() + + f.logger.Trace("forwarder: established UDP connection to %v", id) + go f.proxyUDP(connCtx, pConn, id) } func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack.TransportEndpointID) { @@ -175,11 +205,11 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack errChan := make(chan error, 2) go func() { - errChan <- f.copyUDP(ctx, pConn.conn, pConn.outConn, id, "outbound->inbound") + errChan <- pConn.copy(ctx, pConn.conn, pConn.outConn, &f.udpForwarder.bufPool, "outbound->inbound") }() go func() { - errChan <- f.copyUDP(ctx, pConn.outConn, pConn.conn, id, "inbound->outbound") + errChan <- pConn.copy(ctx, pConn.outConn, pConn.conn, &f.udpForwarder.bufPool, "inbound->outbound") }() select { @@ -193,9 +223,18 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack } } -func (f *Forwarder) copyUDP(ctx context.Context, dst net.Conn, src net.Conn, id stack.TransportEndpointID, direction string) error { - bufp := f.udpForwarder.bufPool.Get().(*[]byte) - defer f.udpForwarder.bufPool.Put(bufp) +func (c *udpPacketConn) updateLastSeen() { + c.lastSeen.Store(time.Now().UnixNano()) +} + +func (c *udpPacketConn) getIdleDuration() time.Duration { + lastSeen := time.Unix(0, c.lastSeen.Load()) + return time.Since(lastSeen) +} + +func (c *udpPacketConn) copy(ctx context.Context, dst net.Conn, src net.Conn, bufPool *sync.Pool, direction string) error { + bufp := bufPool.Get().(*[]byte) + defer bufPool.Put(bufp) buffer := *bufp if err := src.SetReadDeadline(time.Now().Add(udpTimeout)); err != nil { @@ -223,11 +262,7 @@ func (f *Forwarder) copyUDP(ctx context.Context, dst net.Conn, src net.Conn, id return fmt.Errorf("write to %s: %w", direction, err) } - f.udpForwarder.Lock() - if conn, ok := f.udpForwarder.conns[id]; ok { - conn.lastTime = time.Now() - } - f.udpForwarder.Unlock() + c.updateLastSeen() } } } From 9feaa8d767476895fe47513dbeaa22d9e460d022 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 30 Dec 2024 23:05:11 +0100 Subject: [PATCH 06/59] Add icmp forwarder --- .../firewall/uspfilter/forwarder/forwarder.go | 6 +- client/firewall/uspfilter/forwarder/icmp.go | 95 +++++++++++++++++++ client/firewall/uspfilter/forwarder/udp.go | 1 - 3 files changed, 99 insertions(+), 3 deletions(-) create mode 100644 client/firewall/uspfilter/forwarder/icmp.go diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index f3920065851..b9bd471ef98 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -10,6 +10,7 @@ import ( "gvisor.dev/gvisor/pkg/tcpip/header" "gvisor.dev/gvisor/pkg/tcpip/network/ipv4" "gvisor.dev/gvisor/pkg/tcpip/stack" + "gvisor.dev/gvisor/pkg/tcpip/transport/icmp" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" @@ -37,6 +38,7 @@ func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { TransportProtocols: []stack.TransportProtocolFactory{ tcp.NewProtocol, udp.NewProtocol, + icmp.NewProtocol4, }, HandleLocal: false, }) @@ -101,14 +103,14 @@ func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { cancel: cancel, } - // Set up TCP forwarder tcpForwarder := tcp.NewForwarder(s, receiveWindow, maxInFlight, f.handleTCP) s.SetTransportProtocolHandler(tcp.ProtocolNumber, tcpForwarder.HandlePacket) - // Set up UDP forwarder udpForwarder := udp.NewForwarder(s, f.handleUDP) s.SetTransportProtocolHandler(udp.ProtocolNumber, udpForwarder.HandlePacket) + s.SetTransportProtocolHandler(icmp.ProtocolNumber4, f.handleICMP) + log.Debugf("forwarder: Initialization complete with NIC %d", nicID) return f, nil } diff --git a/client/firewall/uspfilter/forwarder/icmp.go b/client/firewall/uspfilter/forwarder/icmp.go new file mode 100644 index 00000000000..5fb80afb5da --- /dev/null +++ b/client/firewall/uspfilter/forwarder/icmp.go @@ -0,0 +1,95 @@ +package forwarder + +import ( + "context" + "net" + "time" + + "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/stack" +) + +// handleICMP handles ICMP packets from the network stack +func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBufferPtr) bool { + ctx, cancel := context.WithTimeout(f.ctx, 5*time.Second) + defer cancel() + + lc := net.ListenConfig{} + // TODO: support non-root + conn, err := lc.ListenPacket(ctx, "ip4:icmp", "0.0.0.0") + if err != nil { + f.logger.Error("Failed to create ICMP socket for %v: %v", id, err) + return false + } + defer func() { + if err := conn.Close(); err != nil { + f.logger.Debug("Failed to close ICMP socket: %v", err) + } + }() + + dstIP := net.IP(id.LocalAddress.AsSlice()) + dst := &net.IPAddr{IP: dstIP} + + // Get the complete ICMP message (header + data) + fullPacket := stack.PayloadSince(pkt.TransportHeader()) + payload := fullPacket.AsSlice() + + icmpHdr := header.ICMPv4(pkt.TransportHeader().View().AsSlice()) + + // For Echo Requests, send and handle response + if icmpHdr.Type() == header.ICMPv4Echo { + _, err = conn.WriteTo(payload, dst) + if err != nil { + f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) + return false + } + + f.logger.Trace("Forwarded ICMP packet %v type=%v code=%v", + id, icmpHdr.Type(), icmpHdr.Code()) + + return f.handleEchoResponse(conn, id) + } + + // TODO: forward other ICMP types + + return true +} + +func (f *Forwarder) handleEchoResponse(conn net.PacketConn, id stack.TransportEndpointID) bool { + if err := conn.SetReadDeadline(time.Now().Add(5 * time.Second)); err != nil { + f.logger.Error("Failed to set read deadline for ICMP response: %v", err) + return false + } + + response := make([]byte, f.endpoint.mtu) + n, _, err := conn.ReadFrom(response) + if err != nil { + if !isTimeout(err) { + f.logger.Error("Failed to read ICMP response: %v", err) + } + return false + } + + ipHdr := make([]byte, header.IPv4MinimumSize) + ip := header.IPv4(ipHdr) + ip.Encode(&header.IPv4Fields{ + TotalLength: uint16(header.IPv4MinimumSize + n), + TTL: 64, + Protocol: uint8(header.ICMPv4ProtocolNumber), + SrcAddr: id.LocalAddress, + DstAddr: id.RemoteAddress, + }) + ip.SetChecksum(^ip.CalculateChecksum()) + + fullPacket := make([]byte, 0, len(ipHdr)+n) + fullPacket = append(fullPacket, ipHdr...) + fullPacket = append(fullPacket, response[:n]...) + + if err := f.InjectIncomingPacket(fullPacket); err != nil { + f.logger.Error("Failed to inject ICMP response: %v", err) + return false + } + + f.logger.Trace("Forwarded ICMP echo reply for %v", id) + return true +} diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index a6f3ab993dc..cbe86f48655 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -9,7 +9,6 @@ import ( "sync/atomic" "time" - log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" From fb1a10755a1020099ed7adf17f33dbcc5fe6cb6d Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Tue, 31 Dec 2024 14:19:15 +0100 Subject: [PATCH 07/59] Fix lint and test issues --- client/firewall/uspfilter/common/iface.go | 4 +-- client/firewall/uspfilter/conntrack/udp.go | 4 +-- .../firewall/uspfilter/forwarder/forwarder.go | 16 +++++----- client/firewall/uspfilter/forwarder/udp.go | 4 +-- client/firewall/uspfilter/log/log.go | 25 ++-------------- client/firewall/uspfilter/log/ringbuffer.go | 8 ----- client/firewall/uspfilter/uspfilter.go | 4 --- client/firewall/uspfilter/uspfilter_test.go | 9 ++++++ client/iface/device_android.go | 3 ++ client/iface/iface_moc.go | 9 ++++-- client/internal/acl/mocks/iface_mapper.go | 30 +++++++++++++++++++ 11 files changed, 64 insertions(+), 52 deletions(-) diff --git a/client/firewall/uspfilter/common/iface.go b/client/firewall/uspfilter/common/iface.go index 3bb12845773..d44e7950936 100644 --- a/client/firewall/uspfilter/common/iface.go +++ b/client/firewall/uspfilter/common/iface.go @@ -1,7 +1,7 @@ package common import ( - device2 "golang.zx2c4.com/wireguard/device" + wgdevice "golang.zx2c4.com/wireguard/device" "github.com/netbirdio/netbird/client/iface" "github.com/netbirdio/netbird/client/iface/device" @@ -11,6 +11,6 @@ import ( type IFaceMapper interface { SetFilter(device.PacketFilter) error Address() iface.WGAddress - GetWGDevice() *device2.Device + GetWGDevice() *wgdevice.Device GetDevice() *device.FilteredDevice } diff --git a/client/firewall/uspfilter/conntrack/udp.go b/client/firewall/uspfilter/conntrack/udp.go index 63000634968..5e37f5cebe8 100644 --- a/client/firewall/uspfilter/conntrack/udp.go +++ b/client/firewall/uspfilter/conntrack/udp.go @@ -75,7 +75,7 @@ func (t *UDPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, d conn.established.Store(true) t.connections[key] = conn - t.logger.Trace("New UDP connection: %s", conn) + t.logger.Trace("New UDP connection: %v", conn) } t.mutex.Unlock() @@ -127,7 +127,7 @@ func (t *UDPTracker) cleanup() { t.ipPool.Put(conn.DestIP) delete(t.connections, key) - t.logger.Trace("UDP connection timed out: %s", conn) + t.logger.Trace("UDP connection timed out: %v", conn) } } } diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index b9bd471ef98..b3430c085f8 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -55,7 +55,7 @@ func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { } if err := s.CreateNIC(nicID, endpoint); err != nil { - return nil, fmt.Errorf("failed to create NIC: %w", err) + return nil, fmt.Errorf("failed to create NIC: %v", err) } _, bits := iface.Address().Network.Mask.Size() @@ -68,7 +68,7 @@ func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { } if err := s.AddProtocolAddress(nicID, protoAddr, stack.AddressProperties{}); err != nil { - return nil, fmt.Errorf("failed to add protocol address: %w", err) + return nil, fmt.Errorf("failed to add protocol address: %s", err) } defaultSubnet, err := tcpip.NewSubnet( @@ -79,11 +79,11 @@ func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { return nil, fmt.Errorf("creating default subnet: %w", err) } - if s.SetPromiscuousMode(nicID, true); err != nil { - return nil, fmt.Errorf("set promiscuous mode: %w", err) + if err := s.SetPromiscuousMode(nicID, true); err != nil { + return nil, fmt.Errorf("set promiscuous mode: %s", err) } - if s.SetSpoofing(nicID, true); err != nil { - return nil, fmt.Errorf("set spoofing: %w", err) + if err := s.SetSpoofing(nicID, true); err != nil { + return nil, fmt.Errorf("set spoofing: %s", err) } s.SetRouteTable([]tcpip.Route{ @@ -132,7 +132,7 @@ func (f *Forwarder) InjectIncomingPacket(payload []byte) error { } // Stop gracefully shuts down the forwarder -func (f *Forwarder) Stop() error { +func (f *Forwarder) Stop() { f.cancel() if f.udpForwarder != nil { @@ -141,6 +141,4 @@ func (f *Forwarder) Stop() error { f.stack.Close() f.stack.Wait() - - return nil } diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index cbe86f48655..951568cbf70 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -131,7 +131,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { } f.udpForwarder.RLock() - pConn, exists := f.udpForwarder.conns[id] + _, exists := f.udpForwarder.conns[id] f.udpForwarder.RUnlock() if exists { f.logger.Trace("forwarder: existing UDP connection for %v", id) @@ -159,7 +159,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { inConn := gonet.NewUDPConn(f.stack, &wq, ep) connCtx, connCancel := context.WithCancel(f.ctx) - pConn = &udpPacketConn{ + pConn := &udpPacketConn{ conn: inConn, outConn: outConn, cancel: connCancel, diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go index 2e9a4d4b729..85f064c1685 100644 --- a/client/firewall/uspfilter/log/log.go +++ b/client/firewall/uspfilter/log/log.go @@ -42,27 +42,6 @@ var levelStrings = map[Level]string{ LevelTrace: "TRAC", } -func FromLogrusLevel(level log.Level) Level { - switch level { - case log.TraceLevel: - return LevelTrace - case log.DebugLevel: - return LevelDebug - case log.InfoLevel: - return LevelInfo - case log.WarnLevel: - return LevelWarn - case log.ErrorLevel: - return LevelError - case log.FatalLevel: - return LevelFatal - case log.PanicLevel: - return LevelPanic - default: - return LevelInfo - } -} - // Logger is a high-performance, non-blocking logger type Logger struct { output io.Writer @@ -128,7 +107,7 @@ func (l *Logger) log(level Level, format string, args ...interface{}) { if len(*bufp) > maxMessageSize { *bufp = (*bufp)[:maxMessageSize] } - l.buffer.Write(*bufp) + _, _ = l.buffer.Write(*bufp) l.bufPool.Put(bufp) } @@ -184,7 +163,7 @@ func (l *Logger) worker() { } // Write batch - l.output.Write(buf[:n]) + _, _ = l.output.Write(buf[:n]) } } } diff --git a/client/firewall/uspfilter/log/ringbuffer.go b/client/firewall/uspfilter/log/ringbuffer.go index 48ebe84aee4..dbc8f1289a7 100644 --- a/client/firewall/uspfilter/log/ringbuffer.go +++ b/client/firewall/uspfilter/log/ringbuffer.go @@ -83,11 +83,3 @@ func (r *ringBuffer) Read(p []byte) (n int, err error) { return n, nil } - -// min returns the smaller of two integers -func min(a, b int) int { - if a < b { - return a - } - return b -} diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 55e2063ecba..022a27a7750 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -555,10 +555,6 @@ func (m *Manager) isValidPacket(d *decoder, packetData []byte) bool { return true } -func (m *Manager) isWireguardTraffic(srcIP, dstIP net.IP) bool { - return m.wgNetwork.Contains(srcIP) && m.wgNetwork.Contains(dstIP) -} - func (m *Manager) isValidTrackedConnection(d *decoder, srcIP, dstIP net.IP) bool { switch d.decoded[1] { case layers.LayerTypeTCP: diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index 2d85116d452..14a7efe775b 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -10,6 +10,7 @@ import ( "github.com/google/gopacket" "github.com/google/gopacket/layers" "github.com/stretchr/testify/require" + wgdevice "golang.zx2c4.com/wireguard/device" fw "github.com/netbirdio/netbird/client/firewall/manager" "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" @@ -22,6 +23,14 @@ type IFaceMock struct { AddressFunc func() iface.WGAddress } +func (i *IFaceMock) GetWGDevice() *wgdevice.Device { + return nil +} + +func (i *IFaceMock) GetDevice() *device.FilteredDevice { + return nil +} + func (i *IFaceMock) SetFilter(iface device.PacketFilter) error { if i.SetFilterFunc == nil { return fmt.Errorf("not implemented") diff --git a/client/iface/device_android.go b/client/iface/device_android.go index 3d15080fff4..028f6fa7d78 100644 --- a/client/iface/device_android.go +++ b/client/iface/device_android.go @@ -1,6 +1,8 @@ package iface import ( + wgdevice "golang.zx2c4.com/wireguard/device" + "github.com/netbirdio/netbird/client/iface/bind" "github.com/netbirdio/netbird/client/iface/device" ) @@ -13,4 +15,5 @@ type WGTunDevice interface { DeviceName() string Close() error FilteredDevice() *device.FilteredDevice + Device() *wgdevice.Device } diff --git a/client/iface/iface_moc.go b/client/iface/iface_moc.go index d91a7224ff2..5f57bc82159 100644 --- a/client/iface/iface_moc.go +++ b/client/iface/iface_moc.go @@ -4,6 +4,7 @@ import ( "net" "time" + wgdevice "golang.zx2c4.com/wireguard/device" "golang.zx2c4.com/wireguard/wgctrl/wgtypes" "github.com/netbirdio/netbird/client/iface/bind" @@ -29,6 +30,7 @@ type MockWGIface struct { SetFilterFunc func(filter device.PacketFilter) error GetFilterFunc func() device.PacketFilter GetDeviceFunc func() *device.FilteredDevice + GetWGDeviceFunc func() *wgdevice.Device GetStatsFunc func(peerKey string) (configurer.WGStats, error) GetInterfaceGUIDStringFunc func() (string, error) GetProxyFunc func() wgproxy.Proxy @@ -102,11 +104,14 @@ func (m *MockWGIface) GetDevice() *device.FilteredDevice { return m.GetDeviceFunc() } +func (m *MockWGIface) GetWGDevice() *wgdevice.Device { + return m.GetWGDeviceFunc() +} + func (m *MockWGIface) GetStats(peerKey string) (configurer.WGStats, error) { return m.GetStatsFunc(peerKey) } func (m *MockWGIface) GetProxy() wgproxy.Proxy { - //TODO implement me - panic("implement me") + return m.GetProxyFunc() } diff --git a/client/internal/acl/mocks/iface_mapper.go b/client/internal/acl/mocks/iface_mapper.go index 3ed12b6dd76..08aa4fd5a01 100644 --- a/client/internal/acl/mocks/iface_mapper.go +++ b/client/internal/acl/mocks/iface_mapper.go @@ -8,6 +8,8 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" + wgdevice "golang.zx2c4.com/wireguard/device" + iface "github.com/netbirdio/netbird/client/iface" "github.com/netbirdio/netbird/client/iface/device" ) @@ -90,3 +92,31 @@ func (mr *MockIFaceMapperMockRecorder) SetFilter(arg0 interface{}) *gomock.Call mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetFilter", reflect.TypeOf((*MockIFaceMapper)(nil).SetFilter), arg0) } + +// GetDevice mocks base method. +func (m *MockIFaceMapper) GetDevice() *device.FilteredDevice { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetDevice") + ret0, _ := ret[0].(*device.FilteredDevice) + return ret0 +} + +// GetDevice indicates an expected call of GetDevice. +func (mr *MockIFaceMapperMockRecorder) GetDevice() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetDevice", reflect.TypeOf((*MockIFaceMapper)(nil).GetDevice)) +} + +// GetWGDevice mocks base method. +func (m *MockIFaceMapper) GetWGDevice() *wgdevice.Device { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetWGDevice") + ret0, _ := ret[0].(*wgdevice.Device) + return ret0 +} + +// GetWGDevice indicates an expected call of GetWGDevice. +func (mr *MockIFaceMapperMockRecorder) GetWGDevice() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWGDevice", reflect.TypeOf((*MockIFaceMapper)(nil).GetWGDevice)) +} From 509b4e21320eb4e4f7e855dbf0ad429fdc0925ec Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Tue, 31 Dec 2024 15:54:45 +0100 Subject: [PATCH 08/59] Lower udp timeout and add teardown messages --- client/firewall/uspfilter/forwarder/tcp.go | 7 +++++-- client/firewall/uspfilter/forwarder/udp.go | 4 +++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index 263fffb513b..e805ea491b2 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -7,6 +7,7 @@ import ( "net" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" + "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" "gvisor.dev/gvisor/pkg/waiter" ) @@ -45,10 +46,10 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { f.logger.Trace("forwarder: established TCP connection to %v", id) - go f.proxyTCP(inConn, outConn) + go f.proxyTCP(id, inConn, outConn) } -func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { +func (f *Forwarder) proxyTCP(id stack.TransportEndpointID, inConn *gonet.TCPConn, outConn net.Conn) { defer func() { if err := inConn.Close(); err != nil { f.logger.Error("forwarder: inConn close error: %v", err) @@ -82,11 +83,13 @@ func (f *Forwarder) proxyTCP(inConn *gonet.TCPConn, outConn net.Conn) { select { case <-ctx.Done(): + f.logger.Trace("forwarder: tearing down TCP connection %v due to context done", id) return case err := <-errChan: if err != nil && !isClosedError(err) { f.logger.Error("proxyTCP: copy error: %v", err) } + f.logger.Trace("forwarder: tearing down TCP connection %v", id) return } } diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 951568cbf70..4491b013561 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -18,7 +18,7 @@ import ( ) const ( - udpTimeout = 60 * time.Second + udpTimeout = 30 * time.Second maxPacketSize = 65535 ) @@ -213,11 +213,13 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack select { case <-ctx.Done(): + f.logger.Trace("forwarder: tearing down UDP connection %v due to context done", id) return case err := <-errChan: if err != nil && !isClosedError(err) { f.logger.Error("proxyUDP: copy error: %v", err) } + f.logger.Trace("forwarder: tearing down UDP connection %v", id) return } } From ed22d79f040e15a81d5d749d2177ce0a55cee95c Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 13:39:36 +0100 Subject: [PATCH 09/59] Add more control with env vars, also allow to pass traffic to native firewall --- client/firewall/uspfilter/uspfilter.go | 80 +++++++++++++++++--------- 1 file changed, 52 insertions(+), 28 deletions(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 022a27a7750..ba7b2e8b54c 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -23,12 +23,17 @@ import ( const layerTypeAll = 0 -const EnvDisableConntrack = "NB_DISABLE_CONNTRACK" +const ( + // EnvDisableConntrack disables the stateful filter, replies to outbound traffic won't be allowed. + EnvDisableConntrack = "NB_DISABLE_CONNTRACK" -// TODO: Add env var to disable routing + // EnvDisableUserspaceRouting disables userspace routing, to-be-routed packets will be dropped. + EnvDisableUserspaceRouting = "NB_DISABLE_USERSPACE_ROUTING" -var ( - errRouteNotSupported = fmt.Errorf("route not supported with userspace firewall") + // EnvForceNativeRouter forces forwarding to the native stack (even if doesn't support routing). + // This is useful when routing/firewall setup is done manually instead of by netbird. + // This setting always disables userspace routing and filtering of routed traffic. + EnvForceNativeRouter = "NB_FORCE_NATIVE_ROUTER" ) // RuleSet is a set of rules grouped by a string key @@ -46,7 +51,10 @@ type Manager struct { mutex sync.RWMutex + // indicates whether we forward packets not destined for ourselves routingEnabled bool + // indicates whether we leave forwarding and filtering to the native firewall + nativeRouter bool stateful bool udpTracker *conntrack.UDPTracker @@ -81,6 +89,17 @@ func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall. } mgr.nativeFirewall = nativeFirewall + + forceNativeRouter, _ := strconv.ParseBool(EnvForceNativeRouter) + // if the OS supports routing natively, or it is explicitly requested, then we don't need to filter/route ourselves + if mgr.nativeFirewall != nil && mgr.nativeFirewall.IsServerRouteSupported() || forceNativeRouter { + mgr.nativeRouter = true + mgr.routingEnabled = true + if mgr.forwarder != nil { + mgr.forwarder.Stop() + } + } + return mgr, nil } @@ -106,9 +125,7 @@ func create(iface common.IFaceMapper) (*Manager, error) { routeRules: make(map[string]RouteRule), wgIface: iface, stateful: !disableConntrack, - // TODO: fix - routingEnabled: true, - // TODO: support chaning log level from logrus + // TODO: support changing log level from logrus logger: nblog.NewFromLogrus(log.StandardLogger()), } @@ -121,18 +138,22 @@ func create(iface common.IFaceMapper) (*Manager, error) { m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout, m.logger) } + if disableRouting, _ := strconv.ParseBool(os.Getenv(EnvDisableUserspaceRouting)); disableRouting { + log.Info("userspace routing is disabled") + return m, nil + } + intf := iface.GetWGDevice() if intf == nil { log.Info("forwarding not supported") // Only supported in userspace mode as we need to inject packets back into wireguard directly - // TODO: Check if native firewall can do the job, in that case just forward everything (restores previous behavior) - m.routingEnabled = false } else { var err error m.forwarder, err = forwarder.New(iface, m.logger) if err != nil { log.Errorf("failed to create forwarder: %v", err) - m.routingEnabled = false + } else { + m.routingEnabled = true } } @@ -147,16 +168,14 @@ func (m *Manager) Init(*statemanager.Manager) error { } func (m *Manager) IsServerRouteSupported() bool { - if m.nativeFirewall == nil { - return false - } else { - return true - } + return m.nativeFirewall != nil || m.routingEnabled && m.forwarder != nil } func (m *Manager) AddNatRule(pair firewall.RouterPair) error { if m.nativeFirewall == nil { - return errRouteNotSupported + // userspace routed packets are always SNATed to the inbound direction + // TODO: implement outbound SNAT + return nil } return m.nativeFirewall.AddNatRule(pair) } @@ -164,7 +183,7 @@ func (m *Manager) AddNatRule(pair firewall.RouterPair) error { // RemoveNatRule removes a routing firewall rule func (m *Manager) RemoveNatRule(pair firewall.RouterPair) error { if m.nativeFirewall == nil { - return errRouteNotSupported + return nil } return m.nativeFirewall.RemoveNatRule(pair) } @@ -450,7 +469,8 @@ func (m *Manager) trackICMPOutbound(d *decoder, srcIP, dstIP net.IP) { } } -// dropFilter implements filtering logic for incoming packets +// dropFilter implements filtering logic for incoming packets. +// If it returns true, the packet should be dropped. func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { m.mutex.RLock() defer m.mutex.RUnlock() @@ -469,8 +489,6 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { return true } - isLocal := m.isLocalIP(dstIP) - // For all inbound traffic, first check if it matches a tracked connection. // This must happen before any other filtering because the packets are statefully tracked. if m.stateful && m.isValidTrackedConnection(d, srcIP, dstIP) { @@ -478,7 +496,7 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { } // Handle local traffic - apply peer ACLs - if isLocal { + if m.isLocalIP(dstIP) { drop := m.applyRules(srcIP, packetData, rules, d) if drop { m.logger.Trace("Dropping local packet: src=%s dst=%s rules=denied", @@ -486,23 +504,28 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { } return drop } + return m.handleRoutedTraffic(d, srcIP, dstIP, packetData) +} - // Handle routed traffic - // TODO: Handle replies for [routed network -> netbird peer], we don't need to start the forwarder here - // We might need to apply NAT - // Don't handle routing if not enabled +func (m *Manager) handleRoutedTraffic(d *decoder, srcIP, dstIP net.IP, packetData []byte) bool { + // Drop if routing is disabled if !m.routingEnabled { m.logger.Trace("Dropping routed packet (routing disabled): src=%s dst=%s", srcIP, dstIP) return true } + // Pass to native stack if native router is enabled or forced + if m.nativeRouter { + return false + } + // Get protocol and ports for route ACL check proto := getProtocolFromPacket(d) srcPort, dstPort := getPortsFromPacket(d) // Check route ACLs - if !m.checkRouteACLs(srcIP, dstIP, proto, srcPort, dstPort) { + if !m.routeACLsPass(srcIP, dstIP, proto, srcPort, dstPort) { m.logger.Trace("Dropping routed packet (ACL denied): src=%s:%d dst=%s:%d proto=%v", srcIP, srcPort, dstIP, dstPort, proto) return true @@ -514,7 +537,7 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { m.logger.Error("Failed to inject incoming packet: %v", err) } - // Default: drop + // Forwarded packets shouldn't reach the native stack, hence they won't be visible in a packet capture return true } @@ -655,7 +678,8 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]PeerRule, d *de return false, false } -func (m *Manager) checkRouteACLs(srcIP, dstIP net.IP, proto firewall.Protocol, srcPort, dstPort uint16) bool { +// routeACLsPass returns treu if the packet is allowed by the route ACLs +func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, srcPort, dstPort uint16) bool { m.mutex.RLock() defer m.mutex.RUnlock() From a12a9ac2900c7eb50dff4d6a1b89444b8d9393d4 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 14:59:41 +0100 Subject: [PATCH 10/59] Handle all local IPs --- client/firewall/uspfilter/localip.go | 128 ++++++++++++++++++++++ client/firewall/uspfilter/localip_test.go | 93 ++++++++++++++++ client/firewall/uspfilter/uspfilter.go | 28 +++-- client/internal/engine.go | 14 ++- 4 files changed, 252 insertions(+), 11 deletions(-) create mode 100644 client/firewall/uspfilter/localip.go create mode 100644 client/firewall/uspfilter/localip_test.go diff --git a/client/firewall/uspfilter/localip.go b/client/firewall/uspfilter/localip.go new file mode 100644 index 00000000000..9636e1cc9b8 --- /dev/null +++ b/client/firewall/uspfilter/localip.go @@ -0,0 +1,128 @@ +package uspfilter + +import ( + "fmt" + "net" + "sync" + + log "github.com/sirupsen/logrus" + + "github.com/netbirdio/netbird/client/firewall/uspfilter/common" +) + +type localIPManager struct { + mu sync.RWMutex + + // Use bitmap for IPv4 (32 bits * 2^16 = 8KB memory) + ipv4Bitmap [1 << 16]uint32 +} + +func newLocalIPManager() *localIPManager { + return &localIPManager{} +} + +func (m *localIPManager) setBitmapBit(ip net.IP) { + ipv4 := ip.To4() + if ipv4 == nil { + return + } + high := (uint16(ipv4[0]) << 8) | uint16(ipv4[1]) + low := (uint16(ipv4[2]) << 8) | uint16(ipv4[3]) + m.ipv4Bitmap[high] |= 1 << (low % 32) +} + +func (m *localIPManager) checkBitmapBit(ip net.IP) bool { + ipv4 := ip.To4() + if ipv4 == nil { + return false + } + high := (uint16(ipv4[0]) << 8) | uint16(ipv4[1]) + low := (uint16(ipv4[2]) << 8) | uint16(ipv4[3]) + return (m.ipv4Bitmap[high] & (1 << (low % 32))) != 0 +} + +func (m *localIPManager) processIP(ip net.IP, newIPv4Bitmap *[1 << 16]uint32, ipv4Set map[string]struct{}, ipv4Addresses *[]string) error { + if ipv4 := ip.To4(); ipv4 != nil { + high := (uint16(ipv4[0]) << 8) | uint16(ipv4[1]) + low := (uint16(ipv4[2]) << 8) | uint16(ipv4[3]) + if int(high) >= len(*newIPv4Bitmap) { + return fmt.Errorf("invalid IPv4 address: %s", ip) + } + ipStr := ip.String() + if _, exists := ipv4Set[ipStr]; !exists { + ipv4Set[ipStr] = struct{}{} + *ipv4Addresses = append(*ipv4Addresses, ipStr) + (*newIPv4Bitmap)[high] |= 1 << (low % 32) + } + } + return nil +} + +func (m *localIPManager) processInterface(iface net.Interface, newIPv4Bitmap *[1 << 16]uint32, ipv4Set map[string]struct{}, ipv4Addresses *[]string) { + addrs, err := iface.Addrs() + if err != nil { + log.Debugf("get addresses for interface %s failed: %v", iface.Name, err) + return + } + + for _, addr := range addrs { + var ip net.IP + switch v := addr.(type) { + case *net.IPNet: + ip = v.IP + case *net.IPAddr: + ip = v.IP + default: + continue + } + + if err := m.processIP(ip, newIPv4Bitmap, ipv4Set, ipv4Addresses); err != nil { + log.Debugf("process IP failed: %v", err) + } + } +} + +func (m *localIPManager) UpdateLocalIPs(iface common.IFaceMapper) (err error) { + defer func() { + if r := recover(); r != nil { + err = fmt.Errorf("panic: %v", r) + } + }() + + interfaces, err := net.Interfaces() + if err != nil { + return fmt.Errorf("get interfaces: %w", err) + } + + var newIPv4Bitmap [1 << 16]uint32 + ipv4Set := make(map[string]struct{}) + var ipv4Addresses []string + + if iface != nil { + if err := m.processIP(iface.Address().IP, &newIPv4Bitmap, ipv4Set, &ipv4Addresses); err != nil { + return err + } + } + + for _, intf := range interfaces { + m.processInterface(intf, &newIPv4Bitmap, ipv4Set, &ipv4Addresses) + } + + m.mu.Lock() + m.ipv4Bitmap = newIPv4Bitmap + m.mu.Unlock() + + log.Debugf("Local IPv4 addresses: %v", ipv4Addresses) + return nil +} + +func (m *localIPManager) IsLocalIP(ip net.IP) bool { + m.mu.RLock() + defer m.mu.RUnlock() + + if ipv4 := ip.To4(); ipv4 != nil { + return m.checkBitmapBit(ipv4) + } + + return false +} diff --git a/client/firewall/uspfilter/localip_test.go b/client/firewall/uspfilter/localip_test.go new file mode 100644 index 00000000000..f179fb2e39a --- /dev/null +++ b/client/firewall/uspfilter/localip_test.go @@ -0,0 +1,93 @@ +package uspfilter + +import ( + "net" + "testing" +) + +// MapImplementation is a version using map[string]struct{} +type MapImplementation struct { + localIPs map[string]struct{} +} + +func BenchmarkIPChecks(b *testing.B) { + interfaces := make([]net.IP, 16) + for i := range interfaces { + interfaces[i] = net.IPv4(10, 0, byte(i>>8), byte(i)) + } + + // Setup bitmap version + bitmapManager := &localIPManager{ + ipv4Bitmap: [1 << 16]uint32{}, + } + for _, ip := range interfaces[:8] { // Add half of IPs + bitmapManager.setBitmapBit(ip) + } + + // Setup map version + mapManager := &MapImplementation{ + localIPs: make(map[string]struct{}), + } + for _, ip := range interfaces[:8] { + mapManager.localIPs[ip.String()] = struct{}{} + } + + b.Run("Bitmap_Hit", func(b *testing.B) { + ip := interfaces[4] + b.ResetTimer() + for i := 0; i < b.N; i++ { + bitmapManager.checkBitmapBit(ip) + } + }) + + b.Run("Bitmap_Miss", func(b *testing.B) { + ip := interfaces[12] + b.ResetTimer() + for i := 0; i < b.N; i++ { + bitmapManager.checkBitmapBit(ip) + } + }) + + b.Run("Map_Hit", func(b *testing.B) { + ip := interfaces[4] + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, _ = mapManager.localIPs[ip.String()] + } + }) + + b.Run("Map_Miss", func(b *testing.B) { + ip := interfaces[12] + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, _ = mapManager.localIPs[ip.String()] + } + }) +} + +func BenchmarkWGPosition(b *testing.B) { + wgIP := net.ParseIP("10.10.0.1") + + // Create two managers - one checks WG IP first, other checks it last + b.Run("WG_First", func(b *testing.B) { + bm := &localIPManager{ipv4Bitmap: [1 << 16]uint32{}} + bm.setBitmapBit(wgIP) + b.ResetTimer() + for i := 0; i < b.N; i++ { + bm.checkBitmapBit(wgIP) + } + }) + + b.Run("WG_Last", func(b *testing.B) { + bm := &localIPManager{ipv4Bitmap: [1 << 16]uint32{}} + // Fill with other IPs first + for i := 0; i < 15; i++ { + bm.setBitmapBit(net.IPv4(10, 0, byte(i>>8), byte(i))) + } + bm.setBitmapBit(wgIP) // Add WG IP last + b.ResetTimer() + for i := 0; i < b.N; i++ { + bm.checkBitmapBit(wgIP) + } + }) +} diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index ba7b2e8b54c..99035c4a385 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -55,8 +55,11 @@ type Manager struct { routingEnabled bool // indicates whether we leave forwarding and filtering to the native firewall nativeRouter bool + // indicates whether we track outbound connections + stateful bool + + localipmanager *localIPManager - stateful bool udpTracker *conntrack.UDPTracker icmpTracker *conntrack.ICMPTracker tcpTracker *conntrack.TCPTracker @@ -120,15 +123,20 @@ func create(iface common.IFaceMapper) (*Manager, error) { return d }, }, - outgoingRules: make(map[string]RuleSet), - incomingRules: make(map[string]RuleSet), - routeRules: make(map[string]RouteRule), - wgIface: iface, - stateful: !disableConntrack, + outgoingRules: make(map[string]RuleSet), + incomingRules: make(map[string]RuleSet), + routeRules: make(map[string]RouteRule), + wgIface: iface, + localipmanager: newLocalIPManager(), + stateful: !disableConntrack, // TODO: support changing log level from logrus logger: nblog.NewFromLogrus(log.StandardLogger()), } + if err := m.localipmanager.UpdateLocalIPs(iface); err != nil { + return nil, fmt.Errorf("update local IPs: %w", err) + } + // Only initialize trackers if stateful mode is enabled if disableConntrack { log.Info("conntrack is disabled") @@ -346,9 +354,9 @@ func (m *Manager) DropIncoming(packetData []byte) bool { return m.dropFilter(packetData, m.incomingRules) } -func (m *Manager) isLocalIP(ip net.IP) bool { - // TODO: add other interface IPs and keep track of them - return ip.Equal(m.wgIface.Address().IP) +// UpdateLocalIPs updates the list of local IPs +func (m *Manager) UpdateLocalIPs() error { + return m.localipmanager.UpdateLocalIPs(m.wgIface) } func (m *Manager) processOutgoingHooks(packetData []byte) bool { @@ -496,7 +504,7 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { } // Handle local traffic - apply peer ACLs - if m.isLocalIP(dstIP) { + if m.localipmanager.IsLocalIP(dstIP) { drop := m.applyRules(srcIP, packetData, rules, d) if drop { m.logger.Trace("Dropping local packet: src=%s dst=%s rules=denied", diff --git a/client/internal/engine.go b/client/internal/engine.go index 042d384dc82..584d41195de 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -40,13 +40,13 @@ import ( "github.com/netbirdio/netbird/client/internal/routemanager" "github.com/netbirdio/netbird/client/internal/routemanager/systemops" "github.com/netbirdio/netbird/client/internal/statemanager" + "github.com/netbirdio/netbird/management/domain" semaphoregroup "github.com/netbirdio/netbird/util/semaphore-group" nbssh "github.com/netbirdio/netbird/client/ssh" "github.com/netbirdio/netbird/client/system" nbdns "github.com/netbirdio/netbird/dns" mgm "github.com/netbirdio/netbird/management/client" - "github.com/netbirdio/netbird/management/domain" mgmProto "github.com/netbirdio/netbird/management/proto" auth "github.com/netbirdio/netbird/relay/auth/hmac" relayClient "github.com/netbirdio/netbird/relay/client" @@ -186,6 +186,10 @@ type Peer struct { WgAllowedIps string } +type localIpUpdater interface { + UpdateLocalIPs() error +} + // NewEngine creates a new Connection Engine func NewEngine( clientCtx context.Context, @@ -802,6 +806,14 @@ func (e *Engine) updateNetworkMap(networkMap *mgmProto.NetworkMap) error { e.acl.ApplyFiltering(networkMap) } + if e.firewall != nil { + if localipfw, ok := e.firewall.(localIpUpdater); ok { + if err := localipfw.UpdateLocalIPs(); err != nil { + log.Errorf("failed to update local IPs: %v", err) + } + } + } + // DNS forwarder dnsRouteFeatureFlag := toDNSFeatureFlag(networkMap) dnsRouteDomains := toRouteDomains(e.config.WgPrivateKey.PublicKey().String(), networkMap.GetRoutes()) From 7667886794c7ea09cfb255ece8aa034204779233 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 15:17:53 +0100 Subject: [PATCH 11/59] Add more tcp logging --- client/firewall/uspfilter/conntrack/tcp.go | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/conntrack/tcp.go b/client/firewall/uspfilter/conntrack/tcp.go index a42208b61fe..3f082d32ce8 100644 --- a/client/firewall/uspfilter/conntrack/tcp.go +++ b/client/firewall/uspfilter/conntrack/tcp.go @@ -177,6 +177,9 @@ func (t *TCPTracker) updateState(conn *TCPConnTrack, flags uint8, isOutbound boo if flags&TCPRst != 0 { conn.State = TCPStateClosed conn.SetEstablished(false) + + t.logger.Trace("TCP connection reset: %s:%d -> %s:%d", + conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) return } @@ -233,6 +236,9 @@ func (t *TCPTracker) updateState(conn *TCPConnTrack, flags uint8, isOutbound boo if flags&TCPAck != 0 { conn.State = TCPStateTimeWait // Keep established = false from previous state + + t.logger.Trace("TCP connection closed (simultaneous) - %s:%d -> %s:%d", + conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) } case TCPStateCloseWait: @@ -243,11 +249,17 @@ func (t *TCPTracker) updateState(conn *TCPConnTrack, flags uint8, isOutbound boo case TCPStateLastAck: if flags&TCPAck != 0 { conn.State = TCPStateClosed + + t.logger.Trace("TCP connection gracefully closed: %s:%d -> %s:%d", + conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) } case TCPStateTimeWait: // Stay in TIME-WAIT for 2MSL before transitioning to closed // This is handled by the cleanup routine + + t.logger.Trace("TCP connection completed - %s:%d -> %s:%d", + conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) } } @@ -325,7 +337,7 @@ func (t *TCPTracker) cleanup() { t.ipPool.Put(conn.DestIP) delete(t.connections, key) - t.logger.Trace("Closed TCP connection: %s:%d -> %s:%d", conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) + t.logger.Trace("Cleaned up TCP connection: %s:%d -> %s:%d", conn.SourceIP, conn.SourcePort, conn.DestIP, conn.DestPort) } } } From d85e57e81957ba8cdcd8475a10c2183917ed1651 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 15:18:00 +0100 Subject: [PATCH 12/59] Handle other icmp types in forwarder --- client/firewall/uspfilter/forwarder/icmp.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/icmp.go b/client/firewall/uspfilter/forwarder/icmp.go index 5fb80afb5da..c9fede72445 100644 --- a/client/firewall/uspfilter/forwarder/icmp.go +++ b/client/firewall/uspfilter/forwarder/icmp.go @@ -37,7 +37,8 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf icmpHdr := header.ICMPv4(pkt.TransportHeader().View().AsSlice()) // For Echo Requests, send and handle response - if icmpHdr.Type() == header.ICMPv4Echo { + switch icmpHdr.Type() { + case header.ICMPv4Echo: _, err = conn.WriteTo(payload, dst) if err != nil { f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) @@ -48,9 +49,21 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf id, icmpHdr.Type(), icmpHdr.Code()) return f.handleEchoResponse(conn, id) + case header.ICMPv4EchoReply: + // dont process our own replies + return false + default: + } + + // For other ICMP types (Time Exceeded, Destination Unreachable, etc) + _, err = conn.WriteTo(payload, dst) + if err != nil { + f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) + return false } - // TODO: forward other ICMP types + f.logger.Trace("Forwarded ICMP packet %v type=%v code=%v", + id, icmpHdr.Type(), icmpHdr.Code()) return true } From 657413b8a665fe4165d28b6ed1f450b77b0fbec8 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 15:56:34 +0100 Subject: [PATCH 13/59] Move icmp acceptance logic --- client/firewall/uspfilter/conntrack/icmp.go | 8 +------- client/firewall/uspfilter/uspfilter.go | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/client/firewall/uspfilter/conntrack/icmp.go b/client/firewall/uspfilter/conntrack/icmp.go index 277a4b26ed4..f9f5fad69e9 100644 --- a/client/firewall/uspfilter/conntrack/icmp.go +++ b/client/firewall/uspfilter/conntrack/icmp.go @@ -97,13 +97,7 @@ func (t *ICMPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, id uint16, seq u // IsValidInbound checks if an inbound ICMP Echo Reply matches a tracked request func (t *ICMPTracker) IsValidInbound(srcIP net.IP, dstIP net.IP, id uint16, seq uint16, icmpType uint8) bool { - switch icmpType { - case uint8(layers.ICMPv4TypeDestinationUnreachable), - uint8(layers.ICMPv4TypeTimeExceeded): - return true - case uint8(layers.ICMPv4TypeEchoReply): - // continue processing - default: + if icmpType != uint8(layers.ICMPv4TypeEchoReply) { return false } diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 99035c4a385..93472210c5d 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -620,7 +620,22 @@ func (m *Manager) isValidTrackedConnection(d *decoder, srcIP, dstIP net.IP) bool return false } +// isSpecialICMP returns true if the packet is a special ICMP packet that should be allowed +func (m *Manager) isSpecialICMP(d *decoder) bool { + if d.decoded[1] != layers.LayerTypeICMPv4 { + return false + } + + icmpType := d.icmp4.TypeCode.Type() + return icmpType == layers.ICMPv4TypeDestinationUnreachable || + icmpType == layers.ICMPv4TypeTimeExceeded +} + func (m *Manager) applyRules(srcIP net.IP, packetData []byte, rules map[string]RuleSet, d *decoder) bool { + if m.isSpecialICMP(d) { + return false + } + if filter, ok := validateRule(srcIP, packetData, rules[srcIP.String()], d); ok { return filter } From fa27369b59ff74ac1092cf4d7fa648a3d8963241 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 16:21:03 +0100 Subject: [PATCH 14/59] Fix linter issues --- client/firewall/uspfilter/localip.go | 2 +- client/firewall/uspfilter/localip_test.go | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/localip.go b/client/firewall/uspfilter/localip.go index 9636e1cc9b8..48ffa44f3a5 100644 --- a/client/firewall/uspfilter/localip.go +++ b/client/firewall/uspfilter/localip.go @@ -52,7 +52,7 @@ func (m *localIPManager) processIP(ip net.IP, newIPv4Bitmap *[1 << 16]uint32, ip if _, exists := ipv4Set[ipStr]; !exists { ipv4Set[ipStr] = struct{}{} *ipv4Addresses = append(*ipv4Addresses, ipStr) - (*newIPv4Bitmap)[high] |= 1 << (low % 32) + newIPv4Bitmap[high] |= 1 << (low % 32) } } return nil diff --git a/client/firewall/uspfilter/localip_test.go b/client/firewall/uspfilter/localip_test.go index f179fb2e39a..91c5d52c55e 100644 --- a/client/firewall/uspfilter/localip_test.go +++ b/client/firewall/uspfilter/localip_test.go @@ -52,6 +52,7 @@ func BenchmarkIPChecks(b *testing.B) { ip := interfaces[4] b.ResetTimer() for i := 0; i < b.N; i++ { + // nolint:gosimple _, _ = mapManager.localIPs[ip.String()] } }) @@ -60,6 +61,7 @@ func BenchmarkIPChecks(b *testing.B) { ip := interfaces[12] b.ResetTimer() for i := 0; i < b.N; i++ { + // nolint:gosimple _, _ = mapManager.localIPs[ip.String()] } }) From 2b8092dfad70e9ac31d7349b8edfa5828ac2ad79 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 16:40:36 +0100 Subject: [PATCH 15/59] Close endpoints --- client/firewall/uspfilter/forwarder/tcp.go | 9 +++++--- client/firewall/uspfilter/forwarder/udp.go | 26 +++++++++++++--------- 2 files changed, 22 insertions(+), 13 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index e805ea491b2..bf5320fe1d6 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -6,6 +6,7 @@ import ( "io" "net" + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" @@ -32,6 +33,7 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { ep, epErr := r.CreateEndpoint(&wq) if epErr != nil { + f.logger.Error("forwarder: failed to create TCP endpoint: %v", epErr) if err := outConn.Close(); err != nil { f.logger.Error("forwarder: outConn close error: %v", err) } @@ -44,12 +46,12 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { inConn := gonet.NewTCPConn(&wq, ep) - f.logger.Trace("forwarder: established TCP connection to %v", id) + f.logger.Trace("forwarder: established TCP connection %v", id) - go f.proxyTCP(id, inConn, outConn) + go f.proxyTCP(id, inConn, outConn, ep) } -func (f *Forwarder) proxyTCP(id stack.TransportEndpointID, inConn *gonet.TCPConn, outConn net.Conn) { +func (f *Forwarder) proxyTCP(id stack.TransportEndpointID, inConn *gonet.TCPConn, outConn net.Conn, ep tcpip.Endpoint) { defer func() { if err := inConn.Close(); err != nil { f.logger.Error("forwarder: inConn close error: %v", err) @@ -57,6 +59,7 @@ func (f *Forwarder) proxyTCP(id stack.TransportEndpointID, inConn *gonet.TCPConn if err := outConn.Close(); err != nil { f.logger.Error("forwarder: outConn close error: %v", err) } + ep.Close() }() // Create context for managing the proxy goroutines diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 4491b013561..85094baaddc 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -9,6 +9,7 @@ import ( "sync/atomic" "time" + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/adapters/gonet" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" @@ -27,6 +28,7 @@ type udpPacketConn struct { outConn net.Conn lastSeen atomic.Int64 cancel context.CancelFunc + ep tcpip.Endpoint } type udpForwarder struct { @@ -38,6 +40,11 @@ type udpForwarder struct { cancel context.CancelFunc } +type idleConn struct { + id stack.TransportEndpointID + conn *udpPacketConn +} + func newUDPForwarder(logger *nblog.Logger) *udpForwarder { ctx, cancel := context.WithCancel(context.Background()) f := &udpForwarder{ @@ -85,18 +92,12 @@ func (f *udpForwarder) cleanup() { case <-f.ctx.Done(): return case <-ticker.C: - var idleConns []struct { - id stack.TransportEndpointID - conn *udpPacketConn - } + var idleConns []idleConn f.RLock() for id, conn := range f.conns { if conn.getIdleDuration() > udpTimeout { - idleConns = append(idleConns, struct { - id stack.TransportEndpointID - conn *udpPacketConn - }{id, conn}) + idleConns = append(idleConns, idleConn{id, conn}) } } f.RUnlock() @@ -110,6 +111,8 @@ func (f *udpForwarder) cleanup() { f.logger.Error("forwarder: UDP outConn close error for %v: %v", idle.id, err) } + idle.conn.ep.Close() + f.Lock() delete(f.conns, idle.id) f.Unlock() @@ -163,6 +166,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { conn: inConn, outConn: outConn, cancel: connCancel, + ep: ep, } pConn.updateLastSeen() @@ -183,10 +187,10 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { f.udpForwarder.Unlock() f.logger.Trace("forwarder: established UDP connection to %v", id) - go f.proxyUDP(connCtx, pConn, id) + go f.proxyUDP(connCtx, pConn, id, ep) } -func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack.TransportEndpointID) { +func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack.TransportEndpointID, ep tcpip.Endpoint) { defer func() { pConn.cancel() if err := pConn.conn.Close(); err != nil { @@ -196,6 +200,8 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) } + ep.Close() + f.udpForwarder.Lock() delete(f.udpForwarder.conns, id) f.udpForwarder.Unlock() From 911f86ded86e8c50dbf8f05144b447a6d3f2fbec Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 17:53:07 +0100 Subject: [PATCH 16/59] Support local IPs in netstack mode --- .../firewall/uspfilter/forwarder/forwarder.go | 14 ++++++- client/firewall/uspfilter/forwarder/icmp.go | 2 +- client/firewall/uspfilter/forwarder/tcp.go | 5 +-- client/firewall/uspfilter/forwarder/udp.go | 6 +-- client/firewall/uspfilter/uspfilter.go | 39 +++++++++++++++---- 5 files changed, 50 insertions(+), 16 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index b3430c085f8..ffd40d09810 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -3,6 +3,7 @@ package forwarder import ( "context" "fmt" + "net" log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/buffer" @@ -30,9 +31,11 @@ type Forwarder struct { udpForwarder *udpForwarder ctx context.Context cancel context.CancelFunc + ip net.IP + netstack bool } -func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { +func New(iface common.IFaceMapper, logger *nblog.Logger, netstack bool) (*Forwarder, error) { s := stack.New(stack.Options{ NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, TransportProtocols: []stack.TransportProtocolFactory{ @@ -101,6 +104,8 @@ func New(iface common.IFaceMapper, logger *nblog.Logger) (*Forwarder, error) { udpForwarder: newUDPForwarder(logger), ctx: ctx, cancel: cancel, + netstack: netstack, + ip: iface.Address().IP, } tcpForwarder := tcp.NewForwarder(s, receiveWindow, maxInFlight, f.handleTCP) @@ -142,3 +147,10 @@ func (f *Forwarder) Stop() { f.stack.Close() f.stack.Wait() } + +func (f *Forwarder) determineDialAddr(addr tcpip.Address) net.IP { + if f.netstack && f.ip.Equal(addr.AsSlice()) { + return net.IPv4(127, 0, 0, 1) + } + return addr.AsSlice() +} diff --git a/client/firewall/uspfilter/forwarder/icmp.go b/client/firewall/uspfilter/forwarder/icmp.go index c9fede72445..10019f21f0d 100644 --- a/client/firewall/uspfilter/forwarder/icmp.go +++ b/client/firewall/uspfilter/forwarder/icmp.go @@ -27,7 +27,7 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf } }() - dstIP := net.IP(id.LocalAddress.AsSlice()) + dstIP := f.determineDialAddr(id.LocalAddress) dst := &net.IPAddr{IP: dstIP} // Get the complete ICMP message (header + data) diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index bf5320fe1d6..efe94bae98d 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -17,10 +17,9 @@ import ( func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { id := r.ID() - dstAddr := id.LocalAddress - dstPort := id.LocalPort - dialAddr := fmt.Sprintf("%s:%d", dstAddr.String(), dstPort) + dialAddr := fmt.Sprintf("%s:%d", f.determineDialAddr(id.LocalAddress), id.LocalPort) + f.logger.Trace("forwarder: handling TCP connection %v", id) outConn, err := (&net.Dialer{}).DialContext(f.ctx, "tcp", dialAddr) if err != nil { r.Complete(true) diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 85094baaddc..a5cba9cb4c3 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -125,14 +125,13 @@ func (f *udpForwarder) cleanup() { // handleUDP is called by the UDP forwarder for new packets func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { - id := r.ID() - dstAddr := fmt.Sprintf("%s:%d", id.LocalAddress.String(), id.LocalPort) - if f.ctx.Err() != nil { f.logger.Trace("forwarder: context done, dropping UDP packet") return } + id := r.ID() + f.udpForwarder.RLock() _, exists := f.udpForwarder.conns[id] f.udpForwarder.RUnlock() @@ -141,6 +140,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { return } + dstAddr := fmt.Sprintf("%s:%d", f.determineDialAddr(id.LocalAddress), id.LocalPort) outConn, err := (&net.Dialer{}).DialContext(f.ctx, "udp", dstAddr) if err != nil { f.logger.Debug("forwarder: UDP dial error for %v: %v", id, err) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 93472210c5d..11ef68a4d3a 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -18,6 +18,7 @@ import ( "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" "github.com/netbirdio/netbird/client/firewall/uspfilter/forwarder" nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" + "github.com/netbirdio/netbird/client/iface/netstack" "github.com/netbirdio/netbird/client/internal/statemanager" ) @@ -57,6 +58,8 @@ type Manager struct { nativeRouter bool // indicates whether we track outbound connections stateful bool + // indicates whether wireguards runs in netstack mode + netstack bool localipmanager *localIPManager @@ -130,7 +133,8 @@ func create(iface common.IFaceMapper) (*Manager, error) { localipmanager: newLocalIPManager(), stateful: !disableConntrack, // TODO: support changing log level from logrus - logger: nblog.NewFromLogrus(log.StandardLogger()), + logger: nblog.NewFromLogrus(log.StandardLogger()), + netstack: netstack.IsEnabled(), } if err := m.localipmanager.UpdateLocalIPs(iface); err != nil { @@ -157,7 +161,7 @@ func create(iface common.IFaceMapper) (*Manager, error) { // Only supported in userspace mode as we need to inject packets back into wireguard directly } else { var err error - m.forwarder, err = forwarder.New(iface, m.logger) + m.forwarder, err = forwarder.New(iface, m.logger, m.netstack) if err != nil { log.Errorf("failed to create forwarder: %v", err) } else { @@ -505,16 +509,36 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { // Handle local traffic - apply peer ACLs if m.localipmanager.IsLocalIP(dstIP) { - drop := m.applyRules(srcIP, packetData, rules, d) - if drop { + if m.peerACLsBlock(srcIP, packetData, rules, d) { m.logger.Trace("Dropping local packet: src=%s dst=%s rules=denied", srcIP, dstIP) + return true } - return drop + + // if running in netstack mode we need to pass this to the forwarder + if m.netstack { + m.logger.Trace("Passing local packet to netstack: src=%s dst=%s", srcIP, dstIP) + m.handleNetstackLocalTraffic(packetData) + // don't process this packet further + return true + } + + return false } + return m.handleRoutedTraffic(d, srcIP, dstIP, packetData) } +func (m *Manager) handleNetstackLocalTraffic(packetData []byte) { + if m.forwarder == nil { + return + } + + if err := m.forwarder.InjectIncomingPacket(packetData); err != nil { + m.logger.Error("Failed to inject local packet: %v", err) + } +} + func (m *Manager) handleRoutedTraffic(d *decoder, srcIP, dstIP net.IP, packetData []byte) bool { // Drop if routing is disabled if !m.routingEnabled { @@ -540,8 +564,7 @@ func (m *Manager) handleRoutedTraffic(d *decoder, srcIP, dstIP net.IP, packetDat } // Let forwarder handle the packet if it passed route ACLs - err := m.forwarder.InjectIncomingPacket(packetData) - if err != nil { + if err := m.forwarder.InjectIncomingPacket(packetData); err != nil { m.logger.Error("Failed to inject incoming packet: %v", err) } @@ -631,7 +654,7 @@ func (m *Manager) isSpecialICMP(d *decoder) bool { icmpType == layers.ICMPv4TypeTimeExceeded } -func (m *Manager) applyRules(srcIP net.IP, packetData []byte, rules map[string]RuleSet, d *decoder) bool { +func (m *Manager) peerACLsBlock(srcIP net.IP, packetData []byte, rules map[string]RuleSet, d *decoder) bool { if m.isSpecialICMP(d) { return false } From 568d064089fbd256668d736be70ec69a0ab1ad41 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 18:56:23 +0100 Subject: [PATCH 17/59] Drop certain forwarded icmp packets --- client/firewall/uspfilter/forwarder/icmp.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/icmp.go b/client/firewall/uspfilter/forwarder/icmp.go index 10019f21f0d..e04464dd97f 100644 --- a/client/firewall/uspfilter/forwarder/icmp.go +++ b/client/firewall/uspfilter/forwarder/icmp.go @@ -19,6 +19,8 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf conn, err := lc.ListenPacket(ctx, "ip4:icmp", "0.0.0.0") if err != nil { f.logger.Error("Failed to create ICMP socket for %v: %v", id, err) + + // This will make netstack reply on behalf of the original destination, that's ok for now return false } defer func() { @@ -42,7 +44,7 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf _, err = conn.WriteTo(payload, dst) if err != nil { f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) - return false + return true } f.logger.Trace("Forwarded ICMP packet %v type=%v code=%v", @@ -51,7 +53,7 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf return f.handleEchoResponse(conn, id) case header.ICMPv4EchoReply: // dont process our own replies - return false + return true default: } @@ -59,7 +61,7 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf _, err = conn.WriteTo(payload, dst) if err != nil { f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) - return false + return true } f.logger.Trace("Forwarded ICMP packet %v type=%v code=%v", From e912f2d7c002af2fe16273e5efd4439a7c69e70c Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 18:59:53 +0100 Subject: [PATCH 18/59] Fix double close in logger --- client/firewall/uspfilter/log/log.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go index 85f064c1685..02f1085612a 100644 --- a/client/firewall/uspfilter/log/log.go +++ b/client/firewall/uspfilter/log/log.go @@ -44,11 +44,12 @@ var levelStrings = map[Level]string{ // Logger is a high-performance, non-blocking logger type Logger struct { - output io.Writer - level atomic.Uint32 - buffer *ringBuffer - shutdown chan struct{} - wg sync.WaitGroup + output io.Writer + level atomic.Uint32 + buffer *ringBuffer + shutdown chan struct{} + closeOnce sync.Once + wg sync.WaitGroup // Reusable buffer pool for formatting messages bufPool sync.Pool @@ -170,9 +171,12 @@ func (l *Logger) worker() { // Stop gracefully shuts down the logger func (l *Logger) Stop(ctx context.Context) error { - close(l.shutdown) - done := make(chan struct{}) + + l.closeOnce.Do(func() { + close(l.shutdown) + }) + go func() { l.wg.Wait() close(done) From f772a21f37266163dc60be683b71e670eca0d797 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 19:02:25 +0100 Subject: [PATCH 19/59] Fix log level handling --- client/firewall/uspfilter/log/log.go | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go index 02f1085612a..1b27fd762a7 100644 --- a/client/firewall/uspfilter/log/log.go +++ b/client/firewall/uspfilter/log/log.go @@ -113,33 +113,33 @@ func (l *Logger) log(level Level, format string, args ...interface{}) { l.bufPool.Put(bufp) } -func (l *Logger) Trace(format string, args ...interface{}) { - if l.level.Load() <= uint32(LevelTrace) { - l.log(LevelTrace, format, args...) +func (l *Logger) Error(format string, args ...interface{}) { + if l.level.Load() >= uint32(LevelError) { + l.log(LevelError, format, args...) } } -func (l *Logger) Debug(format string, args ...interface{}) { - if l.level.Load() <= uint32(LevelDebug) { - l.log(LevelDebug, format, args...) +func (l *Logger) Warn(format string, args ...interface{}) { + if l.level.Load() >= uint32(LevelWarn) { + l.log(LevelWarn, format, args...) } } func (l *Logger) Info(format string, args ...interface{}) { - if l.level.Load() <= uint32(LevelInfo) { + if l.level.Load() >= uint32(LevelInfo) { l.log(LevelInfo, format, args...) } } -func (l *Logger) Warn(format string, args ...interface{}) { - if l.level.Load() <= uint32(LevelWarn) { - l.log(LevelWarn, format, args...) +func (l *Logger) Debug(format string, args ...interface{}) { + if l.level.Load() >= uint32(LevelDebug) { + l.log(LevelDebug, format, args...) } } -func (l *Logger) Error(format string, args ...interface{}) { - if l.level.Load() <= uint32(LevelError) { - l.log(LevelError, format, args...) +func (l *Logger) Trace(format string, args ...interface{}) { + if l.level.Load() >= uint32(LevelTrace) { + l.log(LevelTrace, format, args...) } } From 0b9854b2b108782674d6614e537b22e5f5f5a2b3 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 2 Jan 2025 23:34:43 +0100 Subject: [PATCH 20/59] Fix tests --- .../firewall/uspfilter/conntrack/common_test.go | 6 ++++++ .../firewall/uspfilter/conntrack/icmp_test.go | 4 ++-- client/firewall/uspfilter/conntrack/tcp_test.go | 14 +++++++------- client/firewall/uspfilter/conntrack/udp_test.go | 11 ++++++----- client/firewall/uspfilter/uspfilter_test.go | 17 +++++++++++++++-- 5 files changed, 36 insertions(+), 16 deletions(-) diff --git a/client/firewall/uspfilter/conntrack/common_test.go b/client/firewall/uspfilter/conntrack/common_test.go index b885470a35d..79c1f5d8faa 100644 --- a/client/firewall/uspfilter/conntrack/common_test.go +++ b/client/firewall/uspfilter/conntrack/common_test.go @@ -3,8 +3,14 @@ package conntrack import ( "net" "testing" + + "github.com/sirupsen/logrus" + + "github.com/netbirdio/netbird/client/firewall/uspfilter/log" ) +var logger = log.NewFromLogrus(logrus.StandardLogger()) + func BenchmarkIPOperations(b *testing.B) { b.Run("MakeIPAddr", func(b *testing.B) { ip := net.ParseIP("192.168.1.1") diff --git a/client/firewall/uspfilter/conntrack/icmp_test.go b/client/firewall/uspfilter/conntrack/icmp_test.go index e653416f98f..32553c8360f 100644 --- a/client/firewall/uspfilter/conntrack/icmp_test.go +++ b/client/firewall/uspfilter/conntrack/icmp_test.go @@ -7,7 +7,7 @@ import ( func BenchmarkICMPTracker(b *testing.B) { b.Run("TrackOutbound", func(b *testing.B) { - tracker := NewICMPTracker(DefaultICMPTimeout, nil) + tracker := NewICMPTracker(DefaultICMPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -20,7 +20,7 @@ func BenchmarkICMPTracker(b *testing.B) { }) b.Run("IsValidInbound", func(b *testing.B) { - tracker := NewICMPTracker(DefaultICMPTimeout, nil) + tracker := NewICMPTracker(DefaultICMPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") diff --git a/client/firewall/uspfilter/conntrack/tcp_test.go b/client/firewall/uspfilter/conntrack/tcp_test.go index c44e7dfa735..5f4c43915fb 100644 --- a/client/firewall/uspfilter/conntrack/tcp_test.go +++ b/client/firewall/uspfilter/conntrack/tcp_test.go @@ -9,7 +9,7 @@ import ( ) func TestTCPStateMachine(t *testing.T) { - tracker := NewTCPTracker(DefaultTCPTimeout, nil) + tracker := NewTCPTracker(DefaultTCPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("100.64.0.1") @@ -154,7 +154,7 @@ func TestTCPStateMachine(t *testing.T) { t.Run(tt.name, func(t *testing.T) { t.Helper() - tracker = NewTCPTracker(DefaultTCPTimeout, nil) + tracker = NewTCPTracker(DefaultTCPTimeout, logger) tt.test(t) }) } @@ -162,7 +162,7 @@ func TestTCPStateMachine(t *testing.T) { } func TestRSTHandling(t *testing.T) { - tracker := NewTCPTracker(DefaultTCPTimeout, nil) + tracker := NewTCPTracker(DefaultTCPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("100.64.0.1") @@ -233,7 +233,7 @@ func establishConnection(t *testing.T, tracker *TCPTracker, srcIP, dstIP net.IP, func BenchmarkTCPTracker(b *testing.B) { b.Run("TrackOutbound", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout, nil) + tracker := NewTCPTracker(DefaultTCPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -246,7 +246,7 @@ func BenchmarkTCPTracker(b *testing.B) { }) b.Run("IsValidInbound", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout, nil) + tracker := NewTCPTracker(DefaultTCPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -264,7 +264,7 @@ func BenchmarkTCPTracker(b *testing.B) { }) b.Run("ConcurrentAccess", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout, nil) + tracker := NewTCPTracker(DefaultTCPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -287,7 +287,7 @@ func BenchmarkTCPTracker(b *testing.B) { // Benchmark connection cleanup func BenchmarkCleanup(b *testing.B) { b.Run("TCPCleanup", func(b *testing.B) { - tracker := NewTCPTracker(100*time.Millisecond, nil) // Short timeout for testing + tracker := NewTCPTracker(100*time.Millisecond, logger) // Short timeout for testing defer tracker.Close() // Pre-populate with expired connections diff --git a/client/firewall/uspfilter/conntrack/udp_test.go b/client/firewall/uspfilter/conntrack/udp_test.go index 4e42c484f47..ab4e26d4fdd 100644 --- a/client/firewall/uspfilter/conntrack/udp_test.go +++ b/client/firewall/uspfilter/conntrack/udp_test.go @@ -29,7 +29,7 @@ func TestNewUDPTracker(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - tracker := NewUDPTracker(tt.timeout, nil) + tracker := NewUDPTracker(tt.timeout, logger) assert.NotNil(t, tracker) assert.Equal(t, tt.wantTimeout, tracker.timeout) assert.NotNil(t, tracker.connections) @@ -40,7 +40,7 @@ func TestNewUDPTracker(t *testing.T) { } func TestUDPTracker_TrackOutbound(t *testing.T) { - tracker := NewUDPTracker(DefaultUDPTimeout, nil) + tracker := NewUDPTracker(DefaultUDPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.2") @@ -63,7 +63,7 @@ func TestUDPTracker_TrackOutbound(t *testing.T) { } func TestUDPTracker_IsValidInbound(t *testing.T) { - tracker := NewUDPTracker(1*time.Second, nil) + tracker := NewUDPTracker(1*time.Second, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.2") @@ -162,6 +162,7 @@ func TestUDPTracker_Cleanup(t *testing.T) { cleanupTicker: time.NewTicker(cleanupInterval), done: make(chan struct{}), ipPool: NewPreallocatedIPs(), + logger: logger, } // Start cleanup routine @@ -211,7 +212,7 @@ func TestUDPTracker_Cleanup(t *testing.T) { func BenchmarkUDPTracker(b *testing.B) { b.Run("TrackOutbound", func(b *testing.B) { - tracker := NewUDPTracker(DefaultUDPTimeout, nil) + tracker := NewUDPTracker(DefaultUDPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") @@ -224,7 +225,7 @@ func BenchmarkUDPTracker(b *testing.B) { }) b.Run("IsValidInbound", func(b *testing.B) { - tracker := NewUDPTracker(DefaultUDPTimeout, nil) + tracker := NewUDPTracker(DefaultUDPTimeout, logger) defer tracker.Close() srcIP := net.ParseIP("192.168.1.1") diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index 14a7efe775b..a0c0d23c395 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -9,15 +9,19 @@ import ( "github.com/google/gopacket" "github.com/google/gopacket/layers" + "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" wgdevice "golang.zx2c4.com/wireguard/device" fw "github.com/netbirdio/netbird/client/firewall/manager" "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" + "github.com/netbirdio/netbird/client/firewall/uspfilter/log" "github.com/netbirdio/netbird/client/iface" "github.com/netbirdio/netbird/client/iface/device" ) +var logger = log.NewFromLogrus(logrus.StandardLogger()) + type IFaceMock struct { SetFilterFunc func(device.PacketFilter) error AddressFunc func() iface.WGAddress @@ -284,6 +288,15 @@ func TestManagerReset(t *testing.T) { func TestNotMatchByIP(t *testing.T) { ifaceMock := &IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, + AddressFunc: func() iface.WGAddress { + return iface.WGAddress{ + IP: net.ParseIP("100.10.0.100"), + Network: &net.IPNet{ + IP: net.ParseIP("100.10.0.0"), + Mask: net.CIDRMask(16, 32), + }, + } + }, } m, err := Create(ifaceMock) @@ -409,7 +422,7 @@ func TestProcessOutgoingHooks(t *testing.T) { Mask: net.CIDRMask(16, 32), } manager.udpTracker.Close() - manager.udpTracker = conntrack.NewUDPTracker(100*time.Millisecond, nil) + manager.udpTracker = conntrack.NewUDPTracker(100*time.Millisecond, logger) defer func() { require.NoError(t, manager.Reset(nil)) }() @@ -527,7 +540,7 @@ func TestStatefulFirewall_UDPTracking(t *testing.T) { } manager.udpTracker.Close() // Close the existing tracker - manager.udpTracker = conntrack.NewUDPTracker(200*time.Millisecond, nil) + manager.udpTracker = conntrack.NewUDPTracker(200*time.Millisecond, logger) manager.decoders = sync.Pool{ New: func() any { d := &decoder{ From 2930288f2d5d553a366cddf3c47b268eb74979e8 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 00:21:15 +0100 Subject: [PATCH 21/59] Fix test expectation --- client/internal/acl/manager_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/internal/acl/manager_test.go b/client/internal/acl/manager_test.go index 9a766021a45..d66414af9a7 100644 --- a/client/internal/acl/manager_test.go +++ b/client/internal/acl/manager_test.go @@ -49,6 +49,7 @@ func TestDefaultManager(t *testing.T) { IP: ip, Network: network, }).AnyTimes() + ifaceMock.EXPECT().GetWGDevice().Return(nil).AnyTimes() // we receive one rule from the management so for testing purposes ignore it fw, err := firewall.NewFirewall(ifaceMock, nil) @@ -342,6 +343,7 @@ func TestDefaultManagerEnableSSHRules(t *testing.T) { IP: ip, Network: network, }).AnyTimes() + ifaceMock.EXPECT().GetWGDevice().Return(nil).AnyTimes() // we receive one rule from the management so for testing purposes ignore it fw, err := firewall.NewFirewall(ifaceMock, nil) From 88b420da6d9c67bf6eddab87ec25b9e67a50d6ee Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 00:23:35 +0100 Subject: [PATCH 22/59] Remove linux restriction --- client/internal/routemanager/manager.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/client/internal/routemanager/manager.go b/client/internal/routemanager/manager.go index 0d4a21ac461..c3179fedeec 100644 --- a/client/internal/routemanager/manager.go +++ b/client/internal/routemanager/manager.go @@ -382,11 +382,6 @@ func (m *DefaultManager) classifyRoutes(newRoutes []*route.Route) (map[route.ID] haID := newRoute.GetHAUniqueID() if newRoute.Peer == m.pubKey { ownNetworkIDs[haID] = true - // only linux is supported for now - //if runtime.GOOS != "linux" { - // log.Warnf("received a route to manage, but agent doesn't support router mode on %s OS", runtime.GOOS) - // continue - //} newServerRoutesMap[newRoute.ID] = newRoute } } From 0c2fa38e26a94eaec0a8103d253b87e0ee4a3eb1 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 11:27:52 +0100 Subject: [PATCH 23/59] Exclude benchmark from CI --- client/firewall/uspfilter/uspfilter_bench_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/firewall/uspfilter/uspfilter_bench_test.go b/client/firewall/uspfilter/uspfilter_bench_test.go index 3c661e71c70..4ae88559db8 100644 --- a/client/firewall/uspfilter/uspfilter_bench_test.go +++ b/client/firewall/uspfilter/uspfilter_bench_test.go @@ -1,3 +1,5 @@ +//go:build uspbench + package uspfilter import ( From d711172f6732696e091add139295b69d3c24c0d7 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 11:30:55 +0100 Subject: [PATCH 24/59] Fix benchmarks --- client/firewall/uspfilter/conntrack/common_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client/firewall/uspfilter/conntrack/common_test.go b/client/firewall/uspfilter/conntrack/common_test.go index 79c1f5d8faa..152e4e482c0 100644 --- a/client/firewall/uspfilter/conntrack/common_test.go +++ b/client/firewall/uspfilter/conntrack/common_test.go @@ -70,7 +70,7 @@ func BenchmarkAtomicOperations(b *testing.B) { // Memory pressure tests func BenchmarkMemoryPressure(b *testing.B) { b.Run("TCPHighLoad", func(b *testing.B) { - tracker := NewTCPTracker(DefaultTCPTimeout, nil) + tracker := NewTCPTracker(DefaultTCPTimeout, logger) defer tracker.Close() // Generate different IPs @@ -95,7 +95,7 @@ func BenchmarkMemoryPressure(b *testing.B) { }) b.Run("UDPHighLoad", func(b *testing.B) { - tracker := NewUDPTracker(DefaultUDPTimeout, nil) + tracker := NewUDPTracker(DefaultUDPTimeout, logger) defer tracker.Close() // Generate different IPs From 9490e9095b33b183ebfcc781a426595b9a68b997 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 11:50:25 +0100 Subject: [PATCH 25/59] Reduce complexity --- .../firewall/uspfilter/forwarder/endpoint.go | 2 + client/firewall/uspfilter/forwarder/icmp.go | 27 ++++---- client/firewall/uspfilter/uspfilter.go | 62 +++++++++---------- 3 files changed, 46 insertions(+), 45 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/endpoint.go b/client/firewall/uspfilter/forwarder/endpoint.go index c234ca24136..7ff6101d050 100644 --- a/client/firewall/uspfilter/forwarder/endpoint.go +++ b/client/firewall/uspfilter/forwarder/endpoint.go @@ -67,6 +67,7 @@ func (e *endpoint) WritePackets(pkts stack.PacketBufferList) (int, tcpip.Error) } func (e *endpoint) Wait() { + // not required } func (e *endpoint) ARPHardwareType() header.ARPHardwareType { @@ -74,6 +75,7 @@ func (e *endpoint) ARPHardwareType() header.ARPHardwareType { } func (e *endpoint) AddHeader(*stack.PacketBuffer) { + // not required } func (e *endpoint) ParseHeader(*stack.PacketBuffer) bool { diff --git a/client/firewall/uspfilter/forwarder/icmp.go b/client/firewall/uspfilter/forwarder/icmp.go index e04464dd97f..14cdc37be85 100644 --- a/client/firewall/uspfilter/forwarder/icmp.go +++ b/client/firewall/uspfilter/forwarder/icmp.go @@ -41,16 +41,7 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf // For Echo Requests, send and handle response switch icmpHdr.Type() { case header.ICMPv4Echo: - _, err = conn.WriteTo(payload, dst) - if err != nil { - f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) - return true - } - - f.logger.Trace("Forwarded ICMP packet %v type=%v code=%v", - id, icmpHdr.Type(), icmpHdr.Code()) - - return f.handleEchoResponse(conn, id) + return f.handleEchoResponse(icmpHdr, payload, dst, conn, id) case header.ICMPv4EchoReply: // dont process our own replies return true @@ -70,10 +61,18 @@ func (f *Forwarder) handleICMP(id stack.TransportEndpointID, pkt stack.PacketBuf return true } -func (f *Forwarder) handleEchoResponse(conn net.PacketConn, id stack.TransportEndpointID) bool { +func (f *Forwarder) handleEchoResponse(icmpHdr header.ICMPv4, payload []byte, dst *net.IPAddr, conn net.PacketConn, id stack.TransportEndpointID) bool { + if _, err := conn.WriteTo(payload, dst); err != nil { + f.logger.Error("Failed to write ICMP packet for %v: %v", id, err) + return true + } + + f.logger.Trace("Forwarded ICMP packet %v type=%v code=%v", + id, icmpHdr.Type(), icmpHdr.Code()) + if err := conn.SetReadDeadline(time.Now().Add(5 * time.Second)); err != nil { f.logger.Error("Failed to set read deadline for ICMP response: %v", err) - return false + return true } response := make([]byte, f.endpoint.mtu) @@ -82,7 +81,7 @@ func (f *Forwarder) handleEchoResponse(conn net.PacketConn, id stack.TransportEn if !isTimeout(err) { f.logger.Error("Failed to read ICMP response: %v", err) } - return false + return true } ipHdr := make([]byte, header.IPv4MinimumSize) @@ -102,7 +101,7 @@ func (f *Forwarder) handleEchoResponse(conn net.PacketConn, id stack.TransportEn if err := f.InjectIncomingPacket(fullPacket); err != nil { f.logger.Error("Failed to inject ICMP response: %v", err) - return false + return true } f.logger.Trace("Forwarded ICMP echo reply for %v", id) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 11ef68a4d3a..e869b0f4b84 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -732,47 +732,47 @@ func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, sr srcAddr, _ := netip.AddrFromSlice(srcIP) dstAddr, _ := netip.AddrFromSlice(dstIP) - // Default deny if no rules match matched := false - for _, rule := range m.routeRules { - // Check destination - if !rule.destination.Contains(dstAddr) { - continue - } - - // Check if source matches any source prefix - sourceMatched := false - for _, src := range rule.sources { - if src.Contains(srcAddr) { - sourceMatched = true - break + if m.ruleMatches(rule, srcAddr, dstAddr, proto, srcPort, dstPort) { + matched = true + if rule.action == firewall.ActionDrop { + return false } } - if !sourceMatched { - continue - } + } - // Check protocol - if rule.proto != firewall.ProtocolALL && rule.proto != proto { - continue - } + return matched +} - // Check ports if specified - if rule.srcPort != nil && rule.srcPort.Values[0] != int(srcPort) { - continue - } - if rule.dstPort != nil && rule.dstPort.Values[0] != int(dstPort) { - continue - } +func (m *Manager) ruleMatches(rule RouteRule, srcAddr, dstAddr netip.Addr, proto firewall.Protocol, srcPort, dstPort uint16) bool { + if !rule.destination.Contains(dstAddr) { + return false + } - matched = true - if rule.action == firewall.ActionDrop { - return false + sourceMatched := false + for _, src := range rule.sources { + if src.Contains(srcAddr) { + sourceMatched = true + break } } + if !sourceMatched { + return false + } - return matched + if rule.proto != firewall.ProtocolALL && rule.proto != proto { + return false + } + + if rule.srcPort != nil && rule.srcPort.Values[0] != int(srcPort) { + return false + } + if rule.dstPort != nil && rule.dstPort.Values[0] != int(dstPort) { + return false + } + + return true } // SetNetwork of the wireguard interface to which filtering applied From 955b2b98e1ae73fc99f21e5cd576b2754ccdb1f7 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 14:19:59 +0100 Subject: [PATCH 26/59] Complete route ACLs and add tests --- .../firewall/uspfilter/forwarder/endpoint.go | 2 - client/firewall/uspfilter/forwarder/tcp.go | 1 - client/firewall/uspfilter/uspfilter.go | 69 +- .../uspfilter/uspfilter_bench_test.go | 70 ++ .../uspfilter/uspfilter_filter_test.go | 744 ++++++++++++++++++ client/firewall/uspfilter/uspfilter_test.go | 16 +- 6 files changed, 881 insertions(+), 21 deletions(-) create mode 100644 client/firewall/uspfilter/uspfilter_filter_test.go diff --git a/client/firewall/uspfilter/forwarder/endpoint.go b/client/firewall/uspfilter/forwarder/endpoint.go index 7ff6101d050..e8a265c94d5 100644 --- a/client/firewall/uspfilter/forwarder/endpoint.go +++ b/client/firewall/uspfilter/forwarder/endpoint.go @@ -53,8 +53,6 @@ func (e *endpoint) WritePackets(pkts stack.PacketBufferList) (int, tcpip.Error) // Send the packet through WireGuard address := netHeader.DestinationAddress() - - // TODO: handle dest ip addresses outside our network err := e.device.CreateOutboundPacket(data.AsSlice(), address.AsSlice()) if err != nil { e.logger.Error("CreateOutboundPacket: %v", err) diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index efe94bae98d..3cdc4521842 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -19,7 +19,6 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { dialAddr := fmt.Sprintf("%s:%d", f.determineDialAddr(id.LocalAddress), id.LocalPort) - f.logger.Trace("forwarder: handling TCP connection %v", id) outConn, err := (&net.Dialer{}).DialContext(f.ctx, "tcp", dialAddr) if err != nil { r.Complete(true) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index e869b0f4b84..b6be57ff6bb 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -517,7 +517,6 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { // if running in netstack mode we need to pass this to the forwarder if m.netstack { - m.logger.Trace("Passing local packet to netstack: src=%s dst=%s", srcIP, dstIP) m.handleNetstackLocalTraffic(packetData) // don't process this packet further return true @@ -729,20 +728,39 @@ func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, sr m.mutex.RLock() defer m.mutex.RUnlock() - srcAddr, _ := netip.AddrFromSlice(srcIP) - dstAddr, _ := netip.AddrFromSlice(dstIP) + srcAddr := netip.AddrFrom4([4]byte(srcIP.To4())) + dstAddr := netip.AddrFrom4([4]byte(dstIP.To4())) - matched := false for _, rule := range m.routeRules { - if m.ruleMatches(rule, srcAddr, dstAddr, proto, srcPort, dstPort) { - matched = true - if rule.action == firewall.ActionDrop { - return false + if !rule.destination.Contains(dstAddr) { + continue + } + + sourceMatched := false + for _, src := range rule.sources { + if src.Contains(srcAddr) { + sourceMatched = true + break + } + } + if !sourceMatched { + continue + } + + if rule.proto != firewall.ProtocolALL && rule.proto != proto { + continue + } + + if proto == firewall.ProtocolTCP || proto == firewall.ProtocolUDP { + if !m.portsMatch(rule.srcPort, srcPort) || !m.portsMatch(rule.dstPort, dstPort) { + continue } } + + return rule.action == firewall.ActionAccept } - return matched + return false } func (m *Manager) ruleMatches(rule RouteRule, srcAddr, dstAddr netip.Addr, proto firewall.Protocol, srcPort, dstPort uint16) bool { @@ -765,16 +783,39 @@ func (m *Manager) ruleMatches(rule RouteRule, srcAddr, dstAddr netip.Addr, proto return false } - if rule.srcPort != nil && rule.srcPort.Values[0] != int(srcPort) { - return false - } - if rule.dstPort != nil && rule.dstPort.Values[0] != int(dstPort) { - return false + // Port matches for TCP/UDP only + if proto == firewall.ProtocolTCP || proto == firewall.ProtocolUDP { + return m.portsMatch(rule.srcPort, srcPort) && m.portsMatch(rule.dstPort, dstPort) } return true } +// Add to uspfilter.go, replace existing portsMatch method +func (m *Manager) portsMatch(rulePort *firewall.Port, packetPort uint16) bool { + if rulePort == nil || len(rulePort.Values) == 0 { + return true + } + + if rulePort.IsRange { + if len(rulePort.Values) != 2 { + m.logger.Error("Invalid port range configuration: expected 2 values for range") + return false + } + startPort := rulePort.Values[0] + endPort := rulePort.Values[1] + return int(packetPort) >= startPort && int(packetPort) <= endPort + } + + // Handle list of individual ports + for _, p := range rulePort.Values { + if uint16(p) == packetPort { + return true + } + } + return false +} + // SetNetwork of the wireguard interface to which filtering applied func (m *Manager) SetNetwork(network *net.IPNet) { m.wgNetwork = network diff --git a/client/firewall/uspfilter/uspfilter_bench_test.go b/client/firewall/uspfilter/uspfilter_bench_test.go index 4ae88559db8..c3974b492ff 100644 --- a/client/firewall/uspfilter/uspfilter_bench_test.go +++ b/client/firewall/uspfilter/uspfilter_bench_test.go @@ -6,6 +6,7 @@ import ( "fmt" "math/rand" "net" + "net/netip" "os" "strings" "testing" @@ -998,3 +999,72 @@ func generateTCPPacketWithFlags(b *testing.B, srcIP, dstIP net.IP, srcPort, dstP require.NoError(b, gopacket.SerializeLayers(buf, opts, ipv4, tcp, gopacket.Payload("test"))) return buf.Bytes() } + +func BenchmarkRouteACLs(b *testing.B) { + manager := setupRoutedManager(b, "10.10.0.100/16") + + // Add several route rules to simulate real-world scenario + rules := []struct { + sources []netip.Prefix + dest netip.Prefix + proto fw.Protocol + port *fw.Port + }{ + { + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + port: &fw.Port{Values: []int{80, 443}}, + }, + { + sources: []netip.Prefix{ + netip.MustParsePrefix("172.16.0.0/12"), + netip.MustParsePrefix("10.0.0.0/8"), + }, + dest: netip.MustParsePrefix("0.0.0.0/0"), + proto: fw.ProtocolICMP, + }, + { + sources: []netip.Prefix{netip.MustParsePrefix("0.0.0.0/0")}, + dest: netip.MustParsePrefix("192.168.0.0/16"), + proto: fw.ProtocolUDP, + port: &fw.Port{Values: []int{53}}, + }, + } + + for _, r := range rules { + _, err := manager.AddRouteFiltering( + r.sources, + r.dest, + r.proto, + nil, + r.port, + fw.ActionAccept, + ) + if err != nil { + b.Fatal(err) + } + } + + // Test cases that exercise different matching scenarios + cases := []struct { + srcIP string + dstIP string + proto fw.Protocol + dstPort uint16 + }{ + {"100.10.0.1", "192.168.1.100", fw.ProtocolTCP, 443}, // Match first rule + {"172.16.0.1", "8.8.8.8", fw.ProtocolICMP, 0}, // Match second rule + {"1.1.1.1", "192.168.1.53", fw.ProtocolUDP, 53}, // Match third rule + {"192.168.1.1", "10.0.0.1", fw.ProtocolTCP, 8080}, // No match + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + for _, tc := range cases { + srcIP := net.ParseIP(tc.srcIP) + dstIP := net.ParseIP(tc.dstIP) + manager.routeACLsPass(srcIP, dstIP, tc.proto, 0, tc.dstPort) + } + } +} diff --git a/client/firewall/uspfilter/uspfilter_filter_test.go b/client/firewall/uspfilter/uspfilter_filter_test.go new file mode 100644 index 00000000000..abfa4e54d8c --- /dev/null +++ b/client/firewall/uspfilter/uspfilter_filter_test.go @@ -0,0 +1,744 @@ +package uspfilter + +import ( + "net" + "net/netip" + "testing" + + "github.com/golang/mock/gomock" + "github.com/google/gopacket" + "github.com/google/gopacket/layers" + "github.com/stretchr/testify/require" + wgdevice "golang.zx2c4.com/wireguard/device" + + fw "github.com/netbirdio/netbird/client/firewall/manager" + "github.com/netbirdio/netbird/client/iface" + "github.com/netbirdio/netbird/client/iface/device" + "github.com/netbirdio/netbird/client/iface/mocks" +) + +func TestPeerACLFiltering(t *testing.T) { + localIP := net.ParseIP("100.10.0.100") + wgNet := &net.IPNet{ + IP: net.ParseIP("100.10.0.0"), + Mask: net.CIDRMask(16, 32), + } + + ifaceMock := &IFaceMock{ + SetFilterFunc: func(device.PacketFilter) error { return nil }, + AddressFunc: func() iface.WGAddress { + return iface.WGAddress{ + IP: localIP, + Network: wgNet, + } + }, + } + + manager, err := Create(ifaceMock) + require.NoError(t, err) + require.NotNil(t, manager) + + t.Cleanup(func() { + require.NoError(t, manager.Reset(nil)) + }) + + manager.wgNetwork = wgNet + + err = manager.UpdateLocalIPs() + require.NoError(t, err) + + testCases := []struct { + name string + srcIP string + dstIP string + proto fw.Protocol + srcPort uint16 + dstPort uint16 + ruleIP string + ruleProto fw.Protocol + ruleSrcPort *fw.Port + ruleDstPort *fw.Port + ruleAction fw.Action + shouldBeBlocked bool + }{ + { + name: "Allow TCP traffic from WG peer", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolTCP, + ruleDstPort: &fw.Port{Values: []int{443}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Allow UDP traffic from WG peer", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolUDP, + srcPort: 12345, + dstPort: 53, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolUDP, + ruleDstPort: &fw.Port{Values: []int{53}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Allow ICMP traffic from WG peer", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolICMP, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolICMP, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Allow all traffic from WG peer", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolALL, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Allow traffic from non-WG source", + srcIP: "192.168.1.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + ruleIP: "192.168.1.1", + ruleProto: fw.ProtocolTCP, + ruleDstPort: &fw.Port{Values: []int{443}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Allow all traffic with 0.0.0.0 rule", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + ruleIP: "0.0.0.0", + ruleProto: fw.ProtocolALL, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + } + + t.Run("Implicit DROP (no rules)", func(t *testing.T) { + packet := createTestPacket(t, "100.10.0.1", "100.10.0.100", fw.ProtocolTCP, 12345, 443) + isDropped := manager.DropIncoming(packet) + require.True(t, isDropped, "Packet should be dropped when no rules exist") + }) + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + rules, err := manager.AddPeerFiltering( + net.ParseIP(tc.ruleIP), + tc.ruleProto, + tc.ruleSrcPort, + tc.ruleDstPort, + fw.RuleDirectionIN, + tc.ruleAction, + "", + tc.name, + ) + require.NoError(t, err) + require.NotEmpty(t, rules) + + t.Cleanup(func() { + for _, rule := range rules { + require.NoError(t, manager.DeletePeerRule(rule)) + } + }) + + packet := createTestPacket(t, tc.srcIP, tc.dstIP, tc.proto, tc.srcPort, tc.dstPort) + isDropped := manager.DropIncoming(packet) + require.Equal(t, tc.shouldBeBlocked, isDropped) + }) + } +} + +func createTestPacket(t *testing.T, srcIP, dstIP string, proto fw.Protocol, srcPort, dstPort uint16) []byte { + t.Helper() + + buf := gopacket.NewSerializeBuffer() + opts := gopacket.SerializeOptions{ + ComputeChecksums: true, + FixLengths: true, + } + + ipLayer := &layers.IPv4{ + Version: 4, + TTL: 64, + SrcIP: net.ParseIP(srcIP), + DstIP: net.ParseIP(dstIP), + } + + var err error + switch proto { + case fw.ProtocolTCP: + ipLayer.Protocol = layers.IPProtocolTCP + tcp := &layers.TCP{ + SrcPort: layers.TCPPort(srcPort), + DstPort: layers.TCPPort(dstPort), + } + err = tcp.SetNetworkLayerForChecksum(ipLayer) + require.NoError(t, err) + err = gopacket.SerializeLayers(buf, opts, ipLayer, tcp) + + case fw.ProtocolUDP: + ipLayer.Protocol = layers.IPProtocolUDP + udp := &layers.UDP{ + SrcPort: layers.UDPPort(srcPort), + DstPort: layers.UDPPort(dstPort), + } + err = udp.SetNetworkLayerForChecksum(ipLayer) + require.NoError(t, err) + err = gopacket.SerializeLayers(buf, opts, ipLayer, udp) + + case fw.ProtocolICMP: + ipLayer.Protocol = layers.IPProtocolICMPv4 + icmp := &layers.ICMPv4{ + TypeCode: layers.CreateICMPv4TypeCode(layers.ICMPv4TypeEchoRequest, 0), + } + err = gopacket.SerializeLayers(buf, opts, ipLayer, icmp) + + default: + err = gopacket.SerializeLayers(buf, opts, ipLayer) + } + + require.NoError(t, err) + return buf.Bytes() +} + +func setupRoutedManager(t testing.TB, network string) *Manager { + t.Helper() + + ctrl := gomock.NewController(t) + dev := mocks.NewMockDevice(ctrl) + dev.EXPECT().MTU().Return(1500, nil).AnyTimes() + + localIP, wgNet, err := net.ParseCIDR(network) + require.NoError(t, err) + + ifaceMock := &IFaceMock{ + SetFilterFunc: func(device.PacketFilter) error { return nil }, + AddressFunc: func() iface.WGAddress { + return iface.WGAddress{ + IP: localIP, + Network: wgNet, + } + }, + GetDeviceFunc: func() *device.FilteredDevice { + return &device.FilteredDevice{Device: dev} + }, + GetWGDeviceFunc: func() *wgdevice.Device { + return &wgdevice.Device{} + }, + } + + manager, err := Create(ifaceMock) + require.NoError(t, err) + require.NotNil(t, manager) + require.True(t, manager.routingEnabled) + require.False(t, manager.nativeRouter) + + t.Cleanup(func() { + require.NoError(t, manager.Reset(nil)) + }) + + return manager +} + +func TestRouteACLFiltering(t *testing.T) { + manager := setupRoutedManager(t, "10.10.0.100/16") + + type rule struct { + sources []netip.Prefix + dest netip.Prefix + proto fw.Protocol + srcPort *fw.Port + dstPort *fw.Port + action fw.Action + } + + testCases := []struct { + name string + srcIP string + dstIP string + proto fw.Protocol + srcPort uint16 + dstPort uint16 + rule rule + shouldPass bool + }{ + { + name: "Allow TCP with specific source and destination", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{443}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow any source to specific destination", + srcIP: "172.16.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("0.0.0.0/0")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{443}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow any source to any destination", + srcIP: "172.16.0.1", + dstIP: "203.0.113.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("0.0.0.0/0")}, + dest: netip.MustParsePrefix("0.0.0.0/0"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{443}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow UDP DNS traffic", + srcIP: "100.10.0.1", + dstIP: "192.168.1.53", + proto: fw.ProtocolUDP, + srcPort: 54321, + dstPort: 53, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolUDP, + dstPort: &fw.Port{Values: []int{53}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow ICMP to any destination", + srcIP: "100.10.0.1", + dstIP: "8.8.8.8", + proto: fw.ProtocolICMP, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("0.0.0.0/0"), + proto: fw.ProtocolICMP, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow all protocols but specific port", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolALL, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Implicit deny - wrong destination port", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: false, + }, + { + name: "Implicit deny - wrong protocol", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolUDP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: false, + }, + { + name: "Implicit deny - wrong source network", + srcIP: "172.16.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: false, + }, + { + name: "Source port match", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + srcPort: &fw.Port{Values: []int{12345}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Multiple source networks", + srcIP: "172.16.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{ + netip.MustParsePrefix("100.10.0.0/16"), + netip.MustParsePrefix("172.16.0.0/16"), + }, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow ALL protocol without ports", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolICMP, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolALL, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow ALL protocol with specific ports", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolALL, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Multiple source networks with mismatched protocol", + srcIP: "172.16.0.1", + dstIP: "192.168.1.100", + // Should not match TCP rule + proto: fw.ProtocolUDP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{ + netip.MustParsePrefix("100.10.0.0/16"), + netip.MustParsePrefix("172.16.0.0/16"), + }, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: false, + }, + { + name: "Allow multiple destination ports", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80, 8080, 443}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow multiple source ports", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + srcPort: &fw.Port{Values: []int{12345, 12346, 12347}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Allow ALL protocol with both src and dst ports", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolALL, + srcPort: &fw.Port{Values: []int{12345}}, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Port Range - Within Range", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{ + IsRange: true, + Values: []int{8000, 8100}, + }, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Port Range - Outside Range", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 7999, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{ + IsRange: true, + Values: []int{8000, 8100}, + }, + action: fw.ActionAccept, + }, + shouldPass: false, + }, + { + name: "Source Port Range - Within Range", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 32100, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + srcPort: &fw.Port{ + IsRange: true, + Values: []int{32000, 33000}, + }, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Mixed Port Specification - Range and Single", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 32100, + dstPort: 443, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + srcPort: &fw.Port{ + IsRange: true, + Values: []int{32000, 33000}, + }, + dstPort: &fw.Port{ + Values: []int{443}, + }, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "Invalid Port Range Configuration", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{ + IsRange: true, + Values: []int{8000}, // Invalid: only one value for range + }, + action: fw.ActionAccept, + }, + shouldPass: false, + }, + { + name: "Edge Case - Port at Range Boundary", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8100, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{ + IsRange: true, + Values: []int{8000, 8100}, + }, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "UDP Port Range", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolUDP, + srcPort: 12345, + dstPort: 5060, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolUDP, + dstPort: &fw.Port{ + IsRange: true, + Values: []int{5060, 5070}, + }, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + { + name: "ALL Protocol with Port Range", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolALL, + dstPort: &fw.Port{ + IsRange: true, + Values: []int{8000, 8100}, + }, + action: fw.ActionAccept, + }, + shouldPass: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + rule, err := manager.AddRouteFiltering( + tc.rule.sources, + tc.rule.dest, + tc.rule.proto, + tc.rule.srcPort, + tc.rule.dstPort, + tc.rule.action, + ) + require.NoError(t, err) + require.NotNil(t, rule) + + t.Cleanup(func() { + require.NoError(t, manager.DeleteRouteRule(rule)) + }) + + srcIP := net.ParseIP(tc.srcIP) + dstIP := net.ParseIP(tc.dstIP) + + // testing routeACLsPass only and not DropIncoming, as routed packets are dropped after being passed + // to the forwarder + isAllowed := manager.routeACLsPass(srcIP, dstIP, tc.proto, tc.srcPort, tc.dstPort) + require.Equal(t, tc.shouldPass, isAllowed) + }) + } +} diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index a0c0d23c395..95f79115a8d 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -23,16 +23,24 @@ import ( var logger = log.NewFromLogrus(logrus.StandardLogger()) type IFaceMock struct { - SetFilterFunc func(device.PacketFilter) error - AddressFunc func() iface.WGAddress + SetFilterFunc func(device.PacketFilter) error + AddressFunc func() iface.WGAddress + GetWGDeviceFunc func() *wgdevice.Device + GetDeviceFunc func() *device.FilteredDevice } func (i *IFaceMock) GetWGDevice() *wgdevice.Device { - return nil + if i.GetWGDeviceFunc == nil { + return nil + } + return i.GetWGDeviceFunc() } func (i *IFaceMock) GetDevice() *device.FilteredDevice { - return nil + if i.GetDeviceFunc == nil { + return nil + } + return i.GetDeviceFunc() } func (i *IFaceMock) SetFilter(iface device.PacketFilter) error { From fc799effda37e8b6b40fd54054fe7c451521d07d Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 15:16:30 +0100 Subject: [PATCH 27/59] Set log level from logrus --- client/firewall/uspfilter/log/log.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go index 1b27fd762a7..5e79bebb67c 100644 --- a/client/firewall/uspfilter/log/log.go +++ b/client/firewall/uspfilter/log/log.go @@ -68,7 +68,10 @@ func NewFromLogrus(logrusLogger *log.Logger) *Logger { }, }, } - l.level.Store(uint32(LevelInfo)) + logrusLevel := logrusLogger.GetLevel() + l.level.Store(uint32(logrusLevel)) + level := levelStrings[Level(logrusLevel)] + log.Debugf("New uspfilter logger created with loglevel %v", level) l.wg.Add(1) go l.worker() From c68be6b61b913e11326c1a2a46726ab8acc98a57 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 15:18:36 +0100 Subject: [PATCH 28/59] Remove fractions of seconds --- client/firewall/uspfilter/log/log.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go index 5e79bebb67c..e8e91877ce0 100644 --- a/client/firewall/uspfilter/log/log.go +++ b/client/firewall/uspfilter/log/log.go @@ -87,7 +87,7 @@ func (l *Logger) formatMessage(buf *[]byte, level Level, format string, args ... *buf = (*buf)[:0] // Timestamp - *buf = time.Now().AppendFormat(*buf, "2006-01-02T15:04:05.000000-07:00") + *buf = time.Now().AppendFormat(*buf, "2006-01-02T15:04:05-07:00") *buf = append(*buf, ' ') // Level From 979fe6bb6a9fbbbc03b0c65779513a7238348d7e Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 15:43:28 +0100 Subject: [PATCH 29/59] Reduce complexity and fix linter issues --- client/firewall/uspfilter/uspfilter.go | 33 +++---------------- .../uspfilter/uspfilter_filter_test.go | 20 +++++------ 2 files changed, 15 insertions(+), 38 deletions(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index b6be57ff6bb..49af2854733 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -732,34 +732,10 @@ func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, sr dstAddr := netip.AddrFrom4([4]byte(dstIP.To4())) for _, rule := range m.routeRules { - if !rule.destination.Contains(dstAddr) { - continue - } - - sourceMatched := false - for _, src := range rule.sources { - if src.Contains(srcAddr) { - sourceMatched = true - break - } - } - if !sourceMatched { - continue - } - - if rule.proto != firewall.ProtocolALL && rule.proto != proto { - continue - } - - if proto == firewall.ProtocolTCP || proto == firewall.ProtocolUDP { - if !m.portsMatch(rule.srcPort, srcPort) || !m.portsMatch(rule.dstPort, dstPort) { - continue - } + if m.ruleMatches(rule, srcAddr, dstAddr, proto, srcPort, dstPort) { + return rule.action == firewall.ActionAccept } - - return rule.action == firewall.ActionAccept } - return false } @@ -783,9 +759,10 @@ func (m *Manager) ruleMatches(rule RouteRule, srcAddr, dstAddr netip.Addr, proto return false } - // Port matches for TCP/UDP only if proto == firewall.ProtocolTCP || proto == firewall.ProtocolUDP { - return m.portsMatch(rule.srcPort, srcPort) && m.portsMatch(rule.dstPort, dstPort) + if !m.portsMatch(rule.srcPort, srcPort) || !m.portsMatch(rule.dstPort, dstPort) { + return false + } } return true diff --git a/client/firewall/uspfilter/uspfilter_filter_test.go b/client/firewall/uspfilter/uspfilter_filter_test.go index abfa4e54d8c..93b9471487d 100644 --- a/client/firewall/uspfilter/uspfilter_filter_test.go +++ b/client/firewall/uspfilter/uspfilter_filter_test.go @@ -223,15 +223,15 @@ func createTestPacket(t *testing.T, srcIP, dstIP string, proto fw.Protocol, srcP return buf.Bytes() } -func setupRoutedManager(t testing.TB, network string) *Manager { - t.Helper() +func setupRoutedManager(tb testing.TB, network string) *Manager { + tb.Helper() - ctrl := gomock.NewController(t) + ctrl := gomock.NewController(tb) dev := mocks.NewMockDevice(ctrl) dev.EXPECT().MTU().Return(1500, nil).AnyTimes() localIP, wgNet, err := net.ParseCIDR(network) - require.NoError(t, err) + require.NoError(tb, err) ifaceMock := &IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, @@ -250,13 +250,13 @@ func setupRoutedManager(t testing.TB, network string) *Manager { } manager, err := Create(ifaceMock) - require.NoError(t, err) - require.NotNil(t, manager) - require.True(t, manager.routingEnabled) - require.False(t, manager.nativeRouter) + require.NoError(tb, err) + require.NotNil(tb, manager) + require.True(tb, manager.routingEnabled) + require.False(tb, manager.nativeRouter) - t.Cleanup(func() { - require.NoError(t, manager.Reset(nil)) + tb.Cleanup(func() { + require.NoError(tb, manager.Reset(nil)) }) return manager From f26b418e83b0349824606bc8a9859706581d1944 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 16:02:33 +0100 Subject: [PATCH 30/59] Allow to set firewall log level --- client/firewall/iptables/manager_linux.go | 5 +++++ client/firewall/manager/firewall.go | 2 ++ client/firewall/nftables/manager_linux.go | 5 +++++ client/firewall/uspfilter/log/log.go | 2 ++ client/firewall/uspfilter/uspfilter.go | 7 +++++++ client/internal/engine.go | 5 +++++ client/server/debug.go | 8 ++++++++ 7 files changed, 34 insertions(+) diff --git a/client/firewall/iptables/manager_linux.go b/client/firewall/iptables/manager_linux.go index da8e2c08f7e..8f7084bca04 100644 --- a/client/firewall/iptables/manager_linux.go +++ b/client/firewall/iptables/manager_linux.go @@ -215,6 +215,11 @@ func (m *Manager) AllowNetbird() error { // Flush doesn't need to be implemented for this manager func (m *Manager) Flush() error { return nil } +// SetLogLevel sets the log level for the firewall manager +func (m *Manager) SetLogLevel(log.Level) { + // not supported +} + func getConntrackEstablished() []string { return []string{"-m", "conntrack", "--ctstate", "RELATED,ESTABLISHED", "-j", "ACCEPT"} } diff --git a/client/firewall/manager/firewall.go b/client/firewall/manager/firewall.go index 9391b47ecab..247e55686f6 100644 --- a/client/firewall/manager/firewall.go +++ b/client/firewall/manager/firewall.go @@ -100,6 +100,8 @@ type Manager interface { // Flush the changes to firewall controller Flush() error + + SetLogLevel(log.Level) } func GenKey(format string, pair RouterPair) string { diff --git a/client/firewall/nftables/manager_linux.go b/client/firewall/nftables/manager_linux.go index 8e1aa0d8043..76390d30abd 100644 --- a/client/firewall/nftables/manager_linux.go +++ b/client/firewall/nftables/manager_linux.go @@ -312,6 +312,11 @@ func (m *Manager) cleanupNetbirdTables() error { return nil } +// SetLogLevel sets the log level for the firewall manager +func (m *Manager) SetLogLevel(log.Level) { + // not supported +} + // Flush rule/chain/set operations from the buffer // // Method also get all rules after flush and refreshes handle values in the rulesets diff --git a/client/firewall/uspfilter/log/log.go b/client/firewall/uspfilter/log/log.go index e8e91877ce0..984b6ad08e1 100644 --- a/client/firewall/uspfilter/log/log.go +++ b/client/firewall/uspfilter/log/log.go @@ -81,6 +81,8 @@ func NewFromLogrus(logrusLogger *log.Logger) *Logger { func (l *Logger) SetLevel(level Level) { l.level.Store(uint32(level)) + + log.Debugf("Set uspfilter logger loglevel to %v", levelStrings[level]) } func (l *Logger) formatMessage(buf *[]byte, level Level, format string, args ...interface{}) { diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 49af2854733..96de44ee26b 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -858,3 +858,10 @@ func (m *Manager) RemovePacketHook(hookID string) error { } return fmt.Errorf("hook with given id not found") } + +// SetLogLevel sets the log level for the firewall manager +func (m *Manager) SetLogLevel(level log.Level) { + if m.logger != nil { + m.logger.SetLevel(nblog.Level(level)) + } +} diff --git a/client/internal/engine.go b/client/internal/engine.go index 7cc9f2f2b51..8a7596f0cc1 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -1394,6 +1394,11 @@ func (e *Engine) GetRouteManager() routemanager.Manager { return e.routeManager } +// GetFirewallManager returns the firewall manager +func (e *Engine) GetFirewallManager() manager.Manager { + return e.firewall +} + func findIPFromInterfaceName(ifaceName string) (net.IP, error) { iface, err := net.InterfaceByName(ifaceName) if err != nil { diff --git a/client/server/debug.go b/client/server/debug.go index 3c4967b4e41..6bcf788d1a7 100644 --- a/client/server/debug.go +++ b/client/server/debug.go @@ -488,7 +488,15 @@ func (s *Server) SetLogLevel(_ context.Context, req *proto.SetLogLevelRequest) ( } log.SetLevel(level) + + if s.connectClient != nil && + s.connectClient.Engine() != nil && + s.connectClient.Engine().GetFirewallManager() != nil { + s.connectClient.Engine().GetFirewallManager().SetLogLevel(level) + } + log.Infof("Log level set to %s", level.String()) + return &proto.SetLogLevelResponse{}, nil } From a6ad4dcf22599967452f2310d0805f7e0ccb5c81 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 16:40:51 +0100 Subject: [PATCH 31/59] Close endpoint when stopping udp forwarder --- client/firewall/uspfilter/forwarder/udp.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index a5cba9cb4c3..3861607e07e 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -78,6 +78,8 @@ func (f *udpForwarder) Stop() { if err := conn.outConn.Close(); err != nil { f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) } + + conn.ep.Close() delete(f.conns, id) } } From 62a20f5f1a423f3ffa60ecf08553f17e327f5c78 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 16:50:00 +0100 Subject: [PATCH 32/59] Add local IPs test --- client/firewall/uspfilter/localip_test.go | 139 ++++++++++++++++++++++ 1 file changed, 139 insertions(+) diff --git a/client/firewall/uspfilter/localip_test.go b/client/firewall/uspfilter/localip_test.go index 91c5d52c55e..72a10c970cf 100644 --- a/client/firewall/uspfilter/localip_test.go +++ b/client/firewall/uspfilter/localip_test.go @@ -3,8 +3,147 @@ package uspfilter import ( "net" "testing" + + "github.com/stretchr/testify/require" + + "github.com/netbirdio/netbird/client/iface" ) +func TestLocalIPManager(t *testing.T) { + tests := []struct { + name string + setupAddr iface.WGAddress + testIP net.IP + expected bool + }{ + { + name: "Local IP matches", + setupAddr: iface.WGAddress{ + IP: net.ParseIP("192.168.1.1"), + Network: &net.IPNet{ + IP: net.ParseIP("192.168.1.0"), + Mask: net.CIDRMask(24, 32), + }, + }, + testIP: net.ParseIP("192.168.1.1"), + expected: true, + }, + { + name: "Local IP doesn't match", + setupAddr: iface.WGAddress{ + IP: net.ParseIP("192.168.1.1"), + Network: &net.IPNet{ + IP: net.ParseIP("192.168.1.0"), + Mask: net.CIDRMask(24, 32), + }, + }, + testIP: net.ParseIP("192.168.1.2"), + expected: false, + }, + { + name: "IPv6 address", + setupAddr: iface.WGAddress{ + IP: net.ParseIP("fe80::1"), + Network: &net.IPNet{ + IP: net.ParseIP("fe80::"), + Mask: net.CIDRMask(64, 128), + }, + }, + testIP: net.ParseIP("fe80::1"), + expected: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + manager := newLocalIPManager() + + mock := &IFaceMock{ + AddressFunc: func() iface.WGAddress { + return tt.setupAddr + }, + } + + err := manager.UpdateLocalIPs(mock) + require.NoError(t, err) + + result := manager.IsLocalIP(tt.testIP) + require.Equal(t, tt.expected, result) + }) + } +} + +func TestLocalIPManager_AllInterfaces(t *testing.T) { + manager := newLocalIPManager() + mock := &IFaceMock{} + + // Get actual local interfaces + interfaces, err := net.Interfaces() + require.NoError(t, err) + + var tests []struct { + ip string + expected bool + } + + // Add all local interface IPs to test cases + for _, iface := range interfaces { + addrs, err := iface.Addrs() + require.NoError(t, err) + + for _, addr := range addrs { + var ip net.IP + switch v := addr.(type) { + case *net.IPNet: + ip = v.IP + case *net.IPAddr: + ip = v.IP + default: + continue + } + + if ip4 := ip.To4(); ip4 != nil { + tests = append(tests, struct { + ip string + expected bool + }{ + ip: ip4.String(), + expected: true, + }) + } + } + } + + // Add some external IPs as negative test cases + externalIPs := []string{ + "8.8.8.8", + "1.1.1.1", + "208.67.222.222", + } + for _, ip := range externalIPs { + tests = append(tests, struct { + ip string + expected bool + }{ + ip: ip, + expected: false, + }) + } + + require.NotEmpty(t, tests, "No test cases generated") + + err = manager.UpdateLocalIPs(mock) + require.NoError(t, err) + + t.Logf("Testing %d IPs", len(tests)) + for _, tt := range tests { + t.Run(tt.ip, func(t *testing.T) { + result := manager.IsLocalIP(net.ParseIP(tt.ip)) + require.Equal(t, tt.expected, result, "IP: %s", tt.ip) + }) + } +} + // MapImplementation is a version using map[string]struct{} type MapImplementation struct { localIPs map[string]struct{} From f69dd6fb623feb70fd5f9a953e0cecb52d175622 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 16:54:46 +0100 Subject: [PATCH 33/59] Make extra IPs from interfaces optional --- client/firewall/uspfilter/localip.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/client/firewall/uspfilter/localip.go b/client/firewall/uspfilter/localip.go index 48ffa44f3a5..48c45c098f2 100644 --- a/client/firewall/uspfilter/localip.go +++ b/client/firewall/uspfilter/localip.go @@ -89,11 +89,6 @@ func (m *localIPManager) UpdateLocalIPs(iface common.IFaceMapper) (err error) { } }() - interfaces, err := net.Interfaces() - if err != nil { - return fmt.Errorf("get interfaces: %w", err) - } - var newIPv4Bitmap [1 << 16]uint32 ipv4Set := make(map[string]struct{}) var ipv4Addresses []string @@ -104,8 +99,13 @@ func (m *localIPManager) UpdateLocalIPs(iface common.IFaceMapper) (err error) { } } - for _, intf := range interfaces { - m.processInterface(intf, &newIPv4Bitmap, ipv4Set, &ipv4Addresses) + interfaces, err := net.Interfaces() + if err != nil { + log.Warnf("failed to get interfaces: %v", err) + } else { + for _, intf := range interfaces { + m.processInterface(intf, &newIPv4Bitmap, ipv4Set, &ipv4Addresses) + } } m.mu.Lock() From 0b116b3941b8c9a3df428b2cfcc4bfa1eabe713b Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 17:52:36 +0100 Subject: [PATCH 34/59] Use native firewall for nat/firewall operations if available --- client/firewall/uspfilter/uspfilter.go | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 96de44ee26b..f87664a9e86 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -184,20 +184,21 @@ func (m *Manager) IsServerRouteSupported() bool { } func (m *Manager) AddNatRule(pair firewall.RouterPair) error { - if m.nativeFirewall == nil { - // userspace routed packets are always SNATed to the inbound direction - // TODO: implement outbound SNAT - return nil + if m.nativeRouter { + return m.nativeFirewall.AddNatRule(pair) } - return m.nativeFirewall.AddNatRule(pair) + + // userspace routed packets are always SNATed to the inbound direction + // TODO: implement outbound SNAT + return nil } // RemoveNatRule removes a routing firewall rule func (m *Manager) RemoveNatRule(pair firewall.RouterPair) error { - if m.nativeFirewall == nil { - return nil + if m.nativeRouter { + return m.nativeFirewall.RemoveNatRule(pair) } - return m.nativeFirewall.RemoveNatRule(pair) + return nil } // AddPeerFiltering rule to the firewall @@ -278,6 +279,10 @@ func (m *Manager) AddRouteFiltering( dPort *firewall.Port, action firewall.Action, ) (firewall.Rule, error) { + if m.nativeRouter { + return m.nativeFirewall.AddRouteFiltering(sources, destination, proto, sPort, dPort, action) + } + m.mutex.Lock() defer m.mutex.Unlock() @@ -298,6 +303,10 @@ func (m *Manager) AddRouteFiltering( } func (m *Manager) DeleteRouteRule(rule firewall.Rule) error { + if m.nativeRouter { + return m.nativeFirewall.DeleteRouteRule(rule) + } + m.mutex.Lock() defer m.mutex.Unlock() From eaadb75144bff24b07762f010144bbf0dc2a9ad3 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 17:59:13 +0100 Subject: [PATCH 35/59] Add env var to force userspace routing if native routing is available --- client/firewall/uspfilter/uspfilter.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index f87664a9e86..c429c724330 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -35,6 +35,9 @@ const ( // This is useful when routing/firewall setup is done manually instead of by netbird. // This setting always disables userspace routing and filtering of routed traffic. EnvForceNativeRouter = "NB_FORCE_NATIVE_ROUTER" + + // EnvForceUserspaceRouter forces userspace routing even if native routing is available. + EnvForceUserspaceRouter = "NB_FORCE_USERSPACE_ROUTER" ) // RuleSet is a set of rules grouped by a string key @@ -96,7 +99,13 @@ func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall. mgr.nativeFirewall = nativeFirewall + if forceUserspaceRouter, _ := strconv.ParseBool(os.Getenv(EnvForceUserspaceRouter)); forceUserspaceRouter { + log.Info("userspace routing is forced") + return mgr, nil + } + forceNativeRouter, _ := strconv.ParseBool(EnvForceNativeRouter) + // if the OS supports routing natively, or it is explicitly requested, then we don't need to filter/route ourselves if mgr.nativeFirewall != nil && mgr.nativeFirewall.IsServerRouteSupported() || forceNativeRouter { mgr.nativeRouter = true From 7dfe7e426e9e2eab98165c01d209a9e015973bb3 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 18:38:57 +0100 Subject: [PATCH 36/59] Always use userspace routing in netstack mode --- client/firewall/uspfilter/uspfilter.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index c429c724330..6b28a178554 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -107,14 +107,21 @@ func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall. forceNativeRouter, _ := strconv.ParseBool(EnvForceNativeRouter) // if the OS supports routing natively, or it is explicitly requested, then we don't need to filter/route ourselves - if mgr.nativeFirewall != nil && mgr.nativeFirewall.IsServerRouteSupported() || forceNativeRouter { + // netstack mode won't support userspace routing as there is no interface + if forceNativeRouter || + !netstack.IsEnabled() && mgr.nativeFirewall != nil && mgr.nativeFirewall.IsServerRouteSupported() { + mgr.nativeRouter = true mgr.routingEnabled = true if mgr.forwarder != nil { mgr.forwarder.Stop() } + + log.Info("native routing is enabled") + return mgr, nil } + log.Info("userspace routing is enabled") return mgr, nil } From 766e0cccc9332b7c98c3c20b45e21dfefec4a32d Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 3 Jan 2025 20:56:54 +0100 Subject: [PATCH 37/59] Add packet tracer --- client/cmd/trace.go | 137 ++++++ client/firewall/uspfilter/tracer.go | 388 ++++++++++++++++ client/internal/engine.go | 8 + client/proto/daemon.pb.go | 694 ++++++++++++++++++++++------ client/proto/daemon.proto | 35 ++ client/proto/daemon_grpc.pb.go | 36 ++ client/server/trace.go | 119 +++++ 7 files changed, 1288 insertions(+), 129 deletions(-) create mode 100644 client/cmd/trace.go create mode 100644 client/firewall/uspfilter/tracer.go create mode 100644 client/server/trace.go diff --git a/client/cmd/trace.go b/client/cmd/trace.go new file mode 100644 index 00000000000..b2ff1f1b54e --- /dev/null +++ b/client/cmd/trace.go @@ -0,0 +1,137 @@ +package cmd + +import ( + "fmt" + "math/rand" + "strings" + + "github.com/spf13/cobra" + "google.golang.org/grpc/status" + + "github.com/netbirdio/netbird/client/proto" +) + +var traceCmd = &cobra.Command{ + Use: "trace ", + Short: "Trace a packet through the firewall", + Example: ` + netbird debug trace in 192.168.1.10 10.10.0.2 -p tcp --sport 12345 --dport 443 --syn --ack + netbird debug trace out 10.10.0.1 8.8.8.8 -p udp --dport 53 + netbird debug trace in 10.10.0.2 10.10.0.1 -p icmp --type 8 --code 0 + netbird debug trace in 100.64.1.1 self -p tcp --dport 80`, + Args: cobra.ExactArgs(3), + RunE: tracePacket, +} + +func init() { + debugCmd.AddCommand(traceCmd) + + traceCmd.Flags().StringP("protocol", "p", "tcp", "Protocol (tcp/udp/icmp)") + traceCmd.Flags().Uint16("sport", 0, "Source port") + traceCmd.Flags().Uint16("dport", 0, "Destination port") + traceCmd.Flags().Uint8("icmp-type", 0, "ICMP type") + traceCmd.Flags().Uint8("icmp-code", 0, "ICMP code") + traceCmd.Flags().Bool("syn", false, "TCP SYN flag") + traceCmd.Flags().Bool("ack", false, "TCP ACK flag") + traceCmd.Flags().Bool("fin", false, "TCP FIN flag") + traceCmd.Flags().Bool("rst", false, "TCP RST flag") + traceCmd.Flags().Bool("psh", false, "TCP PSH flag") + traceCmd.Flags().Bool("urg", false, "TCP URG flag") +} + +func tracePacket(cmd *cobra.Command, args []string) error { + direction := strings.ToLower(args[0]) + if direction != "in" && direction != "out" { + return fmt.Errorf("invalid direction: use 'in' or 'out'") + } + + protocol := cmd.Flag("protocol").Value.String() + if protocol != "tcp" && protocol != "udp" && protocol != "icmp" { + return fmt.Errorf("invalid protocol: use tcp/udp/icmp") + } + + sport, err := cmd.Flags().GetUint16("sport") + if err != nil { + return fmt.Errorf("invalid source port: %v", err) + } + dport, err := cmd.Flags().GetUint16("dport") + if err != nil { + return fmt.Errorf("invalid destination port: %v", err) + } + + // For TCP/UDP, generate random ephemeral port (49152-65535) if not specified + if protocol != "icmp" { + if sport == 0 { + sport = uint16(rand.Intn(16383) + 49152) + } + if dport == 0 { + dport = uint16(rand.Intn(16383) + 49152) + } + } + + var tcpFlags *proto.TCPFlags + if protocol == "tcp" { + syn, _ := cmd.Flags().GetBool("syn") + ack, _ := cmd.Flags().GetBool("ack") + fin, _ := cmd.Flags().GetBool("fin") + rst, _ := cmd.Flags().GetBool("rst") + psh, _ := cmd.Flags().GetBool("psh") + urg, _ := cmd.Flags().GetBool("urg") + + tcpFlags = &proto.TCPFlags{ + Syn: syn, + Ack: ack, + Fin: fin, + Rst: rst, + Psh: psh, + Urg: urg, + } + } + + icmpType, _ := cmd.Flags().GetUint32("icmp-type") + icmpCode, _ := cmd.Flags().GetUint32("icmp-code") + + conn, err := getClient(cmd) + if err != nil { + return err + } + defer conn.Close() + + client := proto.NewDaemonServiceClient(conn) + resp, err := client.TracePacket(cmd.Context(), &proto.TracePacketRequest{ + SourceIp: args[1], + DestinationIp: args[2], + Protocol: protocol, + SourcePort: uint32(sport), + DestinationPort: uint32(dport), + Direction: direction, + TcpFlags: tcpFlags, + IcmpType: &icmpType, + IcmpCode: &icmpCode, + }) + if err != nil { + return fmt.Errorf("trace failed: %v", status.Convert(err).Message()) + } + + printTrace(cmd, args[1], args[2], protocol, sport, dport, resp) + return nil +} + +func printTrace(cmd *cobra.Command, src, dst, proto string, sport, dport uint16, resp *proto.TracePacketResponse) { + cmd.Printf("Packet trace %s:%d -> %s:%d (%s)\n\n", src, sport, dst, dport, strings.ToUpper(proto)) + + for _, stage := range resp.Stages { + if stage.ForwardingDetails != nil { + cmd.Printf("%s: %s [%s]\n", stage.Name, stage.Message, *stage.ForwardingDetails) + } else { + cmd.Printf("%s: %s\n", stage.Name, stage.Message) + } + } + + disposition := map[bool]string{ + true: "\033[32mALLOWED\033[0m", // Green + false: "\033[31mDENIED\033[0m", // Red + }[resp.FinalDisposition] + + cmd.Printf("\nFinal disposition: %s\n", disposition) +} diff --git a/client/firewall/uspfilter/tracer.go b/client/firewall/uspfilter/tracer.go new file mode 100644 index 00000000000..379b11ec3c6 --- /dev/null +++ b/client/firewall/uspfilter/tracer.go @@ -0,0 +1,388 @@ +package uspfilter + +import ( + "fmt" + "net" + "time" + + "github.com/google/gopacket" + "github.com/google/gopacket/layers" + + fw "github.com/netbirdio/netbird/client/firewall/manager" + "github.com/netbirdio/netbird/client/firewall/uspfilter/conntrack" +) + +type PacketStage int + +const ( + StageReceived PacketStage = iota + StageConntrack + StagePeerACL + StageRouting + StageRouteACL + StageForwarding + StageCompleted +) + +const msgProcessingCompleted = "Processing completed" + +func (s PacketStage) String() string { + return map[PacketStage]string{ + StageReceived: "Received", + StageConntrack: "Connection Tracking", + StagePeerACL: "Peer ACL", + StageRouting: "Routing", + StageRouteACL: "Route ACL", + StageForwarding: "Forwarding", + StageCompleted: "Completed", + }[s] +} + +type ForwarderAction struct { + Action string + RemoteAddr string + Error error +} + +type TraceResult struct { + Timestamp time.Time + Stage PacketStage + Message string + Allowed bool + ForwarderAction *ForwarderAction +} + +type PacketTrace struct { + SourceIP net.IP + DestinationIP net.IP + Protocol string + SourcePort uint16 + DestinationPort uint16 + Direction fw.RuleDirection + Results []TraceResult +} + +type TCPState struct { + SYN bool + ACK bool + FIN bool + RST bool + PSH bool + URG bool +} + +type PacketBuilder struct { + SrcIP net.IP + DstIP net.IP + Protocol fw.Protocol + SrcPort uint16 + DstPort uint16 + ICMPType uint8 + ICMPCode uint8 + Direction fw.RuleDirection + PayloadSize int + TCPState *TCPState +} + +func (t *PacketTrace) AddResult(stage PacketStage, message string, allowed bool) { + t.Results = append(t.Results, TraceResult{ + Timestamp: time.Now(), + Stage: stage, + Message: message, + Allowed: allowed, + }) +} + +func (t *PacketTrace) AddResultWithForwarder(stage PacketStage, message string, allowed bool, action *ForwarderAction) { + t.Results = append(t.Results, TraceResult{ + Timestamp: time.Now(), + Stage: stage, + Message: message, + Allowed: allowed, + ForwarderAction: action, + }) +} + +func (p *PacketBuilder) Build() ([]byte, error) { + ip := p.buildIPLayer() + pktLayers := []gopacket.SerializableLayer{ip} + + transportLayer, err := p.buildTransportLayer(ip) + if err != nil { + return nil, err + } + pktLayers = append(pktLayers, transportLayer...) + + if p.PayloadSize > 0 { + payload := make([]byte, p.PayloadSize) + pktLayers = append(pktLayers, gopacket.Payload(payload)) + } + + return serializePacket(pktLayers) +} + +func (p *PacketBuilder) buildIPLayer() *layers.IPv4 { + return &layers.IPv4{ + Version: 4, + TTL: 64, + Protocol: layers.IPProtocol(getIPProtocolNumber(p.Protocol)), + SrcIP: p.SrcIP, + DstIP: p.DstIP, + } +} + +func (p *PacketBuilder) buildTransportLayer(ip *layers.IPv4) ([]gopacket.SerializableLayer, error) { + switch p.Protocol { + case "tcp": + return p.buildTCPLayer(ip) + case "udp": + return p.buildUDPLayer(ip) + case "icmp": + return p.buildICMPLayer() + default: + return nil, fmt.Errorf("unsupported protocol: %s", p.Protocol) + } +} + +func (p *PacketBuilder) buildTCPLayer(ip *layers.IPv4) ([]gopacket.SerializableLayer, error) { + tcp := &layers.TCP{ + SrcPort: layers.TCPPort(p.SrcPort), + DstPort: layers.TCPPort(p.DstPort), + Window: 65535, + SYN: p.TCPState != nil && p.TCPState.SYN, + ACK: p.TCPState != nil && p.TCPState.ACK, + FIN: p.TCPState != nil && p.TCPState.FIN, + RST: p.TCPState != nil && p.TCPState.RST, + PSH: p.TCPState != nil && p.TCPState.PSH, + URG: p.TCPState != nil && p.TCPState.URG, + } + if err := tcp.SetNetworkLayerForChecksum(ip); err != nil { + return nil, fmt.Errorf("set network layer for TCP checksum: %w", err) + } + return []gopacket.SerializableLayer{tcp}, nil +} + +func (p *PacketBuilder) buildUDPLayer(ip *layers.IPv4) ([]gopacket.SerializableLayer, error) { + udp := &layers.UDP{ + SrcPort: layers.UDPPort(p.SrcPort), + DstPort: layers.UDPPort(p.DstPort), + } + if err := udp.SetNetworkLayerForChecksum(ip); err != nil { + return nil, fmt.Errorf("set network layer for UDP checksum: %w", err) + } + return []gopacket.SerializableLayer{udp}, nil +} + +func (p *PacketBuilder) buildICMPLayer() ([]gopacket.SerializableLayer, error) { + icmp := &layers.ICMPv4{ + TypeCode: layers.CreateICMPv4TypeCode(p.ICMPType, p.ICMPCode), + } + if p.ICMPType == layers.ICMPv4TypeEchoRequest || p.ICMPType == layers.ICMPv4TypeEchoReply { + icmp.Id = uint16(1) + icmp.Seq = uint16(1) + } + return []gopacket.SerializableLayer{icmp}, nil +} + +func serializePacket(layers []gopacket.SerializableLayer) ([]byte, error) { + buf := gopacket.NewSerializeBuffer() + opts := gopacket.SerializeOptions{ + ComputeChecksums: true, + FixLengths: true, + } + if err := gopacket.SerializeLayers(buf, opts, layers...); err != nil { + return nil, fmt.Errorf("serialize packet: %w", err) + } + return buf.Bytes(), nil +} + +func getIPProtocolNumber(protocol fw.Protocol) int { + switch protocol { + case fw.ProtocolTCP: + return int(layers.IPProtocolTCP) + case fw.ProtocolUDP: + return int(layers.IPProtocolUDP) + case fw.ProtocolICMP: + return int(layers.IPProtocolICMPv4) + default: + return 0 + } +} + +func (m *Manager) TracePacketFromBuilder(builder *PacketBuilder) (*PacketTrace, error) { + packetData, err := builder.Build() + if err != nil { + return nil, fmt.Errorf("build packet: %w", err) + } + + return m.TracePacket(packetData, builder.Direction), nil +} + +func (m *Manager) TracePacket(packetData []byte, direction fw.RuleDirection) *PacketTrace { + + d := m.decoders.Get().(*decoder) + defer m.decoders.Put(d) + + trace := &PacketTrace{Direction: direction} + + // Initial packet decoding + if err := d.parser.DecodeLayers(packetData, &d.decoded); err != nil { + trace.AddResult(StageReceived, fmt.Sprintf("Failed to decode packet: %v", err), false) + return trace + } + + // Extract base packet info + srcIP, dstIP := m.extractIPs(d) + trace.SourceIP = srcIP + trace.DestinationIP = dstIP + + // Determine protocol and ports + switch d.decoded[1] { + case layers.LayerTypeTCP: + trace.Protocol = "TCP" + trace.SourcePort = uint16(d.tcp.SrcPort) + trace.DestinationPort = uint16(d.tcp.DstPort) + case layers.LayerTypeUDP: + trace.Protocol = "UDP" + trace.SourcePort = uint16(d.udp.SrcPort) + trace.DestinationPort = uint16(d.udp.DstPort) + case layers.LayerTypeICMPv4: + trace.Protocol = "ICMP" + } + + trace.AddResult(StageReceived, fmt.Sprintf("Received %s packet: %s:%d -> %s:%d", + trace.Protocol, srcIP, trace.SourcePort, dstIP, trace.DestinationPort), true) + + if direction == fw.RuleDirectionOUT { + return m.traceOutbound(packetData, trace) + } + + return m.traceInbound(packetData, trace, d, srcIP, dstIP) +} + +func (m *Manager) traceInbound(packetData []byte, trace *PacketTrace, d *decoder, srcIP net.IP, dstIP net.IP) *PacketTrace { + if m.stateful && m.handleConntrackState(trace, d, srcIP, dstIP) { + return trace + } + + if m.handleLocalDelivery(trace, packetData, d, srcIP, dstIP) { + return trace + } + + if !m.handleRouting(trace) { + return trace + } + + if m.nativeRouter { + return m.handleNativeRouter(trace) + } + + return m.handleRouteACLs(trace, d, srcIP, dstIP) +} + +func (m *Manager) handleConntrackState(trace *PacketTrace, d *decoder, srcIP, dstIP net.IP) bool { + allowed := m.isValidTrackedConnection(d, srcIP, dstIP) + msg := "No existing connection found" + if allowed { + msg = m.buildConntrackStateMessage(d) + trace.AddResult(StageConntrack, msg, true) + trace.AddResult(StageCompleted, "Packet allowed by connection tracking", true) + return true + } + trace.AddResult(StageConntrack, msg, false) + return false +} + +func (m *Manager) buildConntrackStateMessage(d *decoder) string { + msg := "Matched existing connection state" + switch d.decoded[1] { + case layers.LayerTypeTCP: + flags := getTCPFlags(&d.tcp) + msg += fmt.Sprintf(" (TCP Flags: SYN=%v ACK=%v RST=%v FIN=%v)", + flags&conntrack.TCPSyn != 0, + flags&conntrack.TCPAck != 0, + flags&conntrack.TCPRst != 0, + flags&conntrack.TCPFin != 0) + case layers.LayerTypeICMPv4: + msg += fmt.Sprintf(" (ICMP ID=%d, Seq=%d)", d.icmp4.Id, d.icmp4.Seq) + } + return msg +} + +func (m *Manager) handleLocalDelivery(trace *PacketTrace, packetData []byte, d *decoder, srcIP, dstIP net.IP) bool { + if !m.localipmanager.IsLocalIP(dstIP) { + return false + } + + trace.AddResult(StageRouting, "Packet destined for local delivery", true) + blocked := m.peerACLsBlock(srcIP, packetData, m.incomingRules, d) + + msg := "Allowed by peer ACL rules" + if blocked { + msg = "Blocked by peer ACL rules" + } + trace.AddResult(StagePeerACL, msg, !blocked) + + if m.netstack { + m.addForwardingResult(trace, "proxy-local", "127.0.0.1", !blocked) + } + + trace.AddResult(StageCompleted, msgProcessingCompleted, !blocked) + return true +} + +func (m *Manager) handleRouting(trace *PacketTrace) bool { + if !m.routingEnabled { + trace.AddResult(StageRouting, "Routing disabled", false) + trace.AddResult(StageCompleted, "Packet dropped - routing disabled", false) + return false + } + trace.AddResult(StageRouting, "Routing enabled, checking ACLs", true) + return true +} + +func (m *Manager) handleNativeRouter(trace *PacketTrace) *PacketTrace { + trace.AddResult(StageRouteACL, "Using native router, skipping ACL checks", true) + trace.AddResult(StageForwarding, "Forwarding via native router", true) + trace.AddResult(StageCompleted, msgProcessingCompleted, true) + return trace +} + +func (m *Manager) handleRouteACLs(trace *PacketTrace, d *decoder, srcIP, dstIP net.IP) *PacketTrace { + proto := getProtocolFromPacket(d) + srcPort, dstPort := getPortsFromPacket(d) + allowed := m.routeACLsPass(srcIP, dstIP, proto, srcPort, dstPort) + + msg := "Allowed by route ACLs" + if !allowed { + msg = "Blocked by route ACLs" + } + trace.AddResult(StageRouteACL, msg, allowed) + + if allowed && m.forwarder != nil { + m.addForwardingResult(trace, "proxy-remote", fmt.Sprintf("%s:%d", dstIP, dstPort), true) + } + + trace.AddResult(StageCompleted, msgProcessingCompleted, allowed) + return trace +} + +func (m *Manager) addForwardingResult(trace *PacketTrace, action, remoteAddr string, allowed bool) { + fwdAction := &ForwarderAction{ + Action: action, + RemoteAddr: remoteAddr, + } + trace.AddResultWithForwarder(StageForwarding, + fmt.Sprintf("Forwarding to %s", fwdAction.Action), allowed, fwdAction) +} + +func (m *Manager) traceOutbound(packetData []byte, trace *PacketTrace) *PacketTrace { + // will create or update the connection state + dropped := m.processOutgoingHooks(packetData) + if dropped { + trace.AddResult(StageCompleted, "Packet dropped by outgoing hook", false) + } else { + trace.AddResult(StageCompleted, "Packet allowed (outgoing)", true) + } + return trace +} diff --git a/client/internal/engine.go b/client/internal/engine.go index 8a7596f0cc1..09d35bc8b2f 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -1624,6 +1624,14 @@ func (e *Engine) GetLatestNetworkMap() (*mgmProto.NetworkMap, error) { return nm, nil } +// GetWgAddr returns the wireguard address +func (e *Engine) GetWgAddr() net.IP { + if e.wgInterface == nil { + return nil + } + return e.wgInterface.Address().IP +} + // updateDNSForwarder start or stop the DNS forwarder based on the domains and the feature flag func (e *Engine) updateDNSForwarder(enabled bool, domains []string) { if !enabled { diff --git a/client/proto/daemon.pb.go b/client/proto/daemon.pb.go index f0d3021e92b..9731e2444fd 100644 --- a/client/proto/daemon.pb.go +++ b/client/proto/daemon.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.26.0 -// protoc v3.21.9 +// protoc v4.23.4 // source: daemon.proto package proto @@ -2531,6 +2531,330 @@ func (*SetNetworkMapPersistenceResponse) Descriptor() ([]byte, []int) { return file_daemon_proto_rawDescGZIP(), []int{39} } +type TCPFlags struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Syn bool `protobuf:"varint,1,opt,name=syn,proto3" json:"syn,omitempty"` + Ack bool `protobuf:"varint,2,opt,name=ack,proto3" json:"ack,omitempty"` + Fin bool `protobuf:"varint,3,opt,name=fin,proto3" json:"fin,omitempty"` + Rst bool `protobuf:"varint,4,opt,name=rst,proto3" json:"rst,omitempty"` + Psh bool `protobuf:"varint,5,opt,name=psh,proto3" json:"psh,omitempty"` + Urg bool `protobuf:"varint,6,opt,name=urg,proto3" json:"urg,omitempty"` +} + +func (x *TCPFlags) Reset() { + *x = TCPFlags{} + if protoimpl.UnsafeEnabled { + mi := &file_daemon_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TCPFlags) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TCPFlags) ProtoMessage() {} + +func (x *TCPFlags) ProtoReflect() protoreflect.Message { + mi := &file_daemon_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TCPFlags.ProtoReflect.Descriptor instead. +func (*TCPFlags) Descriptor() ([]byte, []int) { + return file_daemon_proto_rawDescGZIP(), []int{40} +} + +func (x *TCPFlags) GetSyn() bool { + if x != nil { + return x.Syn + } + return false +} + +func (x *TCPFlags) GetAck() bool { + if x != nil { + return x.Ack + } + return false +} + +func (x *TCPFlags) GetFin() bool { + if x != nil { + return x.Fin + } + return false +} + +func (x *TCPFlags) GetRst() bool { + if x != nil { + return x.Rst + } + return false +} + +func (x *TCPFlags) GetPsh() bool { + if x != nil { + return x.Psh + } + return false +} + +func (x *TCPFlags) GetUrg() bool { + if x != nil { + return x.Urg + } + return false +} + +type TracePacketRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SourceIp string `protobuf:"bytes,1,opt,name=source_ip,json=sourceIp,proto3" json:"source_ip,omitempty"` + DestinationIp string `protobuf:"bytes,2,opt,name=destination_ip,json=destinationIp,proto3" json:"destination_ip,omitempty"` + Protocol string `protobuf:"bytes,3,opt,name=protocol,proto3" json:"protocol,omitempty"` + SourcePort uint32 `protobuf:"varint,4,opt,name=source_port,json=sourcePort,proto3" json:"source_port,omitempty"` + DestinationPort uint32 `protobuf:"varint,5,opt,name=destination_port,json=destinationPort,proto3" json:"destination_port,omitempty"` + Direction string `protobuf:"bytes,6,opt,name=direction,proto3" json:"direction,omitempty"` + TcpFlags *TCPFlags `protobuf:"bytes,7,opt,name=tcp_flags,json=tcpFlags,proto3,oneof" json:"tcp_flags,omitempty"` + IcmpType *uint32 `protobuf:"varint,8,opt,name=icmp_type,json=icmpType,proto3,oneof" json:"icmp_type,omitempty"` + IcmpCode *uint32 `protobuf:"varint,9,opt,name=icmp_code,json=icmpCode,proto3,oneof" json:"icmp_code,omitempty"` +} + +func (x *TracePacketRequest) Reset() { + *x = TracePacketRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_daemon_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TracePacketRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TracePacketRequest) ProtoMessage() {} + +func (x *TracePacketRequest) ProtoReflect() protoreflect.Message { + mi := &file_daemon_proto_msgTypes[41] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TracePacketRequest.ProtoReflect.Descriptor instead. +func (*TracePacketRequest) Descriptor() ([]byte, []int) { + return file_daemon_proto_rawDescGZIP(), []int{41} +} + +func (x *TracePacketRequest) GetSourceIp() string { + if x != nil { + return x.SourceIp + } + return "" +} + +func (x *TracePacketRequest) GetDestinationIp() string { + if x != nil { + return x.DestinationIp + } + return "" +} + +func (x *TracePacketRequest) GetProtocol() string { + if x != nil { + return x.Protocol + } + return "" +} + +func (x *TracePacketRequest) GetSourcePort() uint32 { + if x != nil { + return x.SourcePort + } + return 0 +} + +func (x *TracePacketRequest) GetDestinationPort() uint32 { + if x != nil { + return x.DestinationPort + } + return 0 +} + +func (x *TracePacketRequest) GetDirection() string { + if x != nil { + return x.Direction + } + return "" +} + +func (x *TracePacketRequest) GetTcpFlags() *TCPFlags { + if x != nil { + return x.TcpFlags + } + return nil +} + +func (x *TracePacketRequest) GetIcmpType() uint32 { + if x != nil && x.IcmpType != nil { + return *x.IcmpType + } + return 0 +} + +func (x *TracePacketRequest) GetIcmpCode() uint32 { + if x != nil && x.IcmpCode != nil { + return *x.IcmpCode + } + return 0 +} + +type TraceStage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` + Allowed bool `protobuf:"varint,3,opt,name=allowed,proto3" json:"allowed,omitempty"` + ForwardingDetails *string `protobuf:"bytes,4,opt,name=forwarding_details,json=forwardingDetails,proto3,oneof" json:"forwarding_details,omitempty"` +} + +func (x *TraceStage) Reset() { + *x = TraceStage{} + if protoimpl.UnsafeEnabled { + mi := &file_daemon_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TraceStage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TraceStage) ProtoMessage() {} + +func (x *TraceStage) ProtoReflect() protoreflect.Message { + mi := &file_daemon_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TraceStage.ProtoReflect.Descriptor instead. +func (*TraceStage) Descriptor() ([]byte, []int) { + return file_daemon_proto_rawDescGZIP(), []int{42} +} + +func (x *TraceStage) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *TraceStage) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *TraceStage) GetAllowed() bool { + if x != nil { + return x.Allowed + } + return false +} + +func (x *TraceStage) GetForwardingDetails() string { + if x != nil && x.ForwardingDetails != nil { + return *x.ForwardingDetails + } + return "" +} + +type TracePacketResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Stages []*TraceStage `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"` + FinalDisposition bool `protobuf:"varint,2,opt,name=final_disposition,json=finalDisposition,proto3" json:"final_disposition,omitempty"` +} + +func (x *TracePacketResponse) Reset() { + *x = TracePacketResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_daemon_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TracePacketResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TracePacketResponse) ProtoMessage() {} + +func (x *TracePacketResponse) ProtoReflect() protoreflect.Message { + mi := &file_daemon_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TracePacketResponse.ProtoReflect.Descriptor instead. +func (*TracePacketResponse) Descriptor() ([]byte, []int) { + return file_daemon_proto_rawDescGZIP(), []int{43} +} + +func (x *TracePacketResponse) GetStages() []*TraceStage { + if x != nil { + return x.Stages + } + return nil +} + +func (x *TracePacketResponse) GetFinalDisposition() bool { + if x != nil { + return x.FinalDisposition + } + return false +} + var File_daemon_proto protoreflect.FileDescriptor var file_daemon_proto_rawDesc = []byte{ @@ -2857,88 +3181,142 @@ var file_daemon_proto_rawDesc = []byte{ 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x22, 0x22, 0x0a, 0x20, 0x53, 0x65, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4d, 0x61, 0x70, 0x50, 0x65, 0x72, 0x73, 0x69, - 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x62, - 0x0a, 0x08, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, - 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x41, 0x4e, 0x49, 0x43, - 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x41, 0x54, 0x41, 0x4c, 0x10, 0x02, 0x12, 0x09, 0x0a, - 0x05, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x03, 0x12, 0x08, 0x0a, 0x04, 0x57, 0x41, 0x52, 0x4e, - 0x10, 0x04, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x4e, 0x46, 0x4f, 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, - 0x44, 0x45, 0x42, 0x55, 0x47, 0x10, 0x06, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x52, 0x41, 0x43, 0x45, - 0x10, 0x07, 0x32, 0x93, 0x09, 0x0a, 0x0d, 0x44, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x12, 0x36, 0x0a, 0x05, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x12, 0x14, 0x2e, - 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x15, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x67, - 0x69, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0c, - 0x57, 0x61, 0x69, 0x74, 0x53, 0x53, 0x4f, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x12, 0x1b, 0x2e, 0x64, - 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x57, 0x61, 0x69, 0x74, 0x53, 0x53, 0x4f, 0x4c, 0x6f, 0x67, - 0x69, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x64, 0x61, 0x65, 0x6d, + 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x76, + 0x0a, 0x08, 0x54, 0x43, 0x50, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x79, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x73, 0x79, 0x6e, 0x12, 0x10, 0x0a, 0x03, + 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x12, 0x10, + 0x0a, 0x03, 0x66, 0x69, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x66, 0x69, 0x6e, + 0x12, 0x10, 0x0a, 0x03, 0x72, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x72, + 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x03, 0x70, 0x73, 0x68, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x03, 0x75, 0x72, 0x67, 0x22, 0x80, 0x03, 0x0a, 0x12, 0x54, 0x72, 0x61, 0x63, 0x65, + 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, + 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x70, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x65, + 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x70, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x70, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x1f, 0x0a, + 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x29, + 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x6f, + 0x72, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x69, + 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x0a, 0x09, 0x74, 0x63, 0x70, 0x5f, 0x66, + 0x6c, 0x61, 0x67, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x64, 0x61, 0x65, + 0x6d, 0x6f, 0x6e, 0x2e, 0x54, 0x43, 0x50, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x48, 0x00, 0x52, 0x08, + 0x74, 0x63, 0x70, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x69, + 0x63, 0x6d, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0d, 0x48, 0x01, + 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, + 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, + 0x48, 0x02, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x43, 0x6f, 0x64, 0x65, 0x88, 0x01, 0x01, 0x42, + 0x0c, 0x0a, 0x0a, 0x5f, 0x74, 0x63, 0x70, 0x5f, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x42, 0x0c, 0x0a, + 0x0a, 0x5f, 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, + 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x9f, 0x01, 0x0a, 0x0a, 0x54, 0x72, + 0x61, 0x63, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, + 0x12, 0x32, 0x0a, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x64, + 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, + 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x88, 0x01, 0x01, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x6e, 0x0a, 0x13, 0x54, + 0x72, 0x61, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x54, 0x72, 0x61, 0x63, + 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x12, 0x2b, + 0x0a, 0x11, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x5f, 0x64, 0x69, 0x73, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x66, 0x69, 0x6e, 0x61, 0x6c, + 0x44, 0x69, 0x73, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x2a, 0x62, 0x0a, 0x08, 0x4c, + 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, + 0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x41, 0x4e, 0x49, 0x43, 0x10, 0x01, 0x12, + 0x09, 0x0a, 0x05, 0x46, 0x41, 0x54, 0x41, 0x4c, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, + 0x52, 0x4f, 0x52, 0x10, 0x03, 0x12, 0x08, 0x0a, 0x04, 0x57, 0x41, 0x52, 0x4e, 0x10, 0x04, 0x12, + 0x08, 0x0a, 0x04, 0x49, 0x4e, 0x46, 0x4f, 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x45, 0x42, + 0x55, 0x47, 0x10, 0x06, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x52, 0x41, 0x43, 0x45, 0x10, 0x07, 0x32, + 0xdd, 0x09, 0x0a, 0x0d, 0x44, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 0x36, 0x0a, 0x05, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x12, 0x14, 0x2e, 0x64, 0x61, 0x65, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x15, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0c, 0x57, 0x61, 0x69, + 0x74, 0x53, 0x53, 0x4f, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x12, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x57, 0x61, 0x69, 0x74, 0x53, 0x53, 0x4f, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x2d, 0x0a, 0x02, 0x55, 0x70, 0x12, - 0x11, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x12, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x39, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x15, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x64, 0x61, 0x65, 0x6d, - 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x33, 0x0a, 0x04, 0x44, 0x6f, 0x77, 0x6e, 0x12, 0x13, 0x2e, 0x64, 0x61, - 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x14, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x42, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x47, - 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x19, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0c, - 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x1b, 0x2e, 0x64, - 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, - 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x64, 0x61, 0x65, 0x6d, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, + 0x57, 0x61, 0x69, 0x74, 0x53, 0x53, 0x4f, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x2d, 0x0a, 0x02, 0x55, 0x70, 0x12, 0x11, 0x2e, 0x64, + 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x12, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x39, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x15, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x33, 0x0a, 0x04, 0x44, 0x6f, 0x77, 0x6e, 0x12, 0x13, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, + 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, + 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x42, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x18, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x64, + 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0c, 0x4c, 0x69, 0x73, + 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0e, 0x53, 0x65, 0x6c, - 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x1d, 0x2e, 0x64, 0x61, - 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, - 0x72, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x64, 0x61, 0x65, - 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, - 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x10, - 0x44, 0x65, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, - 0x12, 0x1d, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1e, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, - 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x44, 0x65, 0x62, 0x75, 0x67, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, - 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x42, - 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x64, - 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x42, 0x75, 0x6e, 0x64, 0x6c, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x47, - 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, - 0x6d, 0x6f, 0x6e, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, - 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, - 0x65, 0x76, 0x65, 0x6c, 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, - 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, - 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x45, 0x0a, 0x0a, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x2e, - 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, - 0x6e, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x45, 0x0a, 0x0a, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x12, 0x19, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6c, - 0x65, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, - 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1a, 0x2e, 0x64, - 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, - 0x6e, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x18, 0x53, 0x65, 0x74, 0x4e, 0x65, - 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4d, 0x61, 0x70, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, - 0x6e, 0x63, 0x65, 0x12, 0x27, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, - 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4d, 0x61, 0x70, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, - 0x74, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x64, - 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, - 0x4d, 0x61, 0x70, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x08, 0x5a, 0x06, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, + 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x1d, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x10, 0x44, 0x65, 0x73, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x1d, 0x2e, + 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, + 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x64, + 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, + 0x6f, 0x72, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, + 0x0a, 0x0b, 0x44, 0x65, 0x62, 0x75, 0x67, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x1a, 0x2e, + 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x42, 0x75, 0x6e, 0x64, + 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, + 0x6f, 0x6e, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4c, + 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, + 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x47, 0x65, 0x74, + 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, + 0x6c, 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4c, 0x6f, + 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, + 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, + 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x45, 0x0a, 0x0a, + 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x19, 0x2e, 0x64, 0x61, 0x65, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x45, 0x0a, 0x0a, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x19, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6c, 0x65, 0x61, 0x6e, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x64, + 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, + 0x6f, 0x6e, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x18, 0x53, 0x65, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, + 0x72, 0x6b, 0x4d, 0x61, 0x70, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, + 0x12, 0x27, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x65, 0x74, + 0x77, 0x6f, 0x72, 0x6b, 0x4d, 0x61, 0x70, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, + 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x64, 0x61, 0x65, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4d, 0x61, 0x70, + 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x50, 0x61, + 0x63, 0x6b, 0x65, 0x74, 0x12, 0x1a, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x54, 0x72, + 0x61, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1b, 0x2e, 0x64, 0x61, 0x65, 0x6d, 0x6f, 0x6e, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x50, + 0x61, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, + 0x08, 0x5a, 0x06, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, } var ( @@ -2954,7 +3332,7 @@ func file_daemon_proto_rawDescGZIP() []byte { } var file_daemon_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_daemon_proto_msgTypes = make([]protoimpl.MessageInfo, 41) +var file_daemon_proto_msgTypes = make([]protoimpl.MessageInfo, 45) var file_daemon_proto_goTypes = []interface{}{ (LogLevel)(0), // 0: daemon.LogLevel (*LoginRequest)(nil), // 1: daemon.LoginRequest @@ -2997,16 +3375,20 @@ var file_daemon_proto_goTypes = []interface{}{ (*DeleteStateResponse)(nil), // 38: daemon.DeleteStateResponse (*SetNetworkMapPersistenceRequest)(nil), // 39: daemon.SetNetworkMapPersistenceRequest (*SetNetworkMapPersistenceResponse)(nil), // 40: daemon.SetNetworkMapPersistenceResponse - nil, // 41: daemon.Network.ResolvedIPsEntry - (*durationpb.Duration)(nil), // 42: google.protobuf.Duration - (*timestamppb.Timestamp)(nil), // 43: google.protobuf.Timestamp + (*TCPFlags)(nil), // 41: daemon.TCPFlags + (*TracePacketRequest)(nil), // 42: daemon.TracePacketRequest + (*TraceStage)(nil), // 43: daemon.TraceStage + (*TracePacketResponse)(nil), // 44: daemon.TracePacketResponse + nil, // 45: daemon.Network.ResolvedIPsEntry + (*durationpb.Duration)(nil), // 46: google.protobuf.Duration + (*timestamppb.Timestamp)(nil), // 47: google.protobuf.Timestamp } var file_daemon_proto_depIdxs = []int32{ - 42, // 0: daemon.LoginRequest.dnsRouteInterval:type_name -> google.protobuf.Duration + 46, // 0: daemon.LoginRequest.dnsRouteInterval:type_name -> google.protobuf.Duration 19, // 1: daemon.StatusResponse.fullStatus:type_name -> daemon.FullStatus - 43, // 2: daemon.PeerState.connStatusUpdate:type_name -> google.protobuf.Timestamp - 43, // 3: daemon.PeerState.lastWireguardHandshake:type_name -> google.protobuf.Timestamp - 42, // 4: daemon.PeerState.latency:type_name -> google.protobuf.Duration + 47, // 2: daemon.PeerState.connStatusUpdate:type_name -> google.protobuf.Timestamp + 47, // 3: daemon.PeerState.lastWireguardHandshake:type_name -> google.protobuf.Timestamp + 46, // 4: daemon.PeerState.latency:type_name -> google.protobuf.Duration 16, // 5: daemon.FullStatus.managementState:type_name -> daemon.ManagementState 15, // 6: daemon.FullStatus.signalState:type_name -> daemon.SignalState 14, // 7: daemon.FullStatus.localPeerState:type_name -> daemon.LocalPeerState @@ -3014,48 +3396,52 @@ var file_daemon_proto_depIdxs = []int32{ 17, // 9: daemon.FullStatus.relays:type_name -> daemon.RelayState 18, // 10: daemon.FullStatus.dns_servers:type_name -> daemon.NSGroupState 25, // 11: daemon.ListNetworksResponse.routes:type_name -> daemon.Network - 41, // 12: daemon.Network.resolvedIPs:type_name -> daemon.Network.ResolvedIPsEntry + 45, // 12: daemon.Network.resolvedIPs:type_name -> daemon.Network.ResolvedIPsEntry 0, // 13: daemon.GetLogLevelResponse.level:type_name -> daemon.LogLevel 0, // 14: daemon.SetLogLevelRequest.level:type_name -> daemon.LogLevel 32, // 15: daemon.ListStatesResponse.states:type_name -> daemon.State - 24, // 16: daemon.Network.ResolvedIPsEntry.value:type_name -> daemon.IPList - 1, // 17: daemon.DaemonService.Login:input_type -> daemon.LoginRequest - 3, // 18: daemon.DaemonService.WaitSSOLogin:input_type -> daemon.WaitSSOLoginRequest - 5, // 19: daemon.DaemonService.Up:input_type -> daemon.UpRequest - 7, // 20: daemon.DaemonService.Status:input_type -> daemon.StatusRequest - 9, // 21: daemon.DaemonService.Down:input_type -> daemon.DownRequest - 11, // 22: daemon.DaemonService.GetConfig:input_type -> daemon.GetConfigRequest - 20, // 23: daemon.DaemonService.ListNetworks:input_type -> daemon.ListNetworksRequest - 22, // 24: daemon.DaemonService.SelectNetworks:input_type -> daemon.SelectNetworksRequest - 22, // 25: daemon.DaemonService.DeselectNetworks:input_type -> daemon.SelectNetworksRequest - 26, // 26: daemon.DaemonService.DebugBundle:input_type -> daemon.DebugBundleRequest - 28, // 27: daemon.DaemonService.GetLogLevel:input_type -> daemon.GetLogLevelRequest - 30, // 28: daemon.DaemonService.SetLogLevel:input_type -> daemon.SetLogLevelRequest - 33, // 29: daemon.DaemonService.ListStates:input_type -> daemon.ListStatesRequest - 35, // 30: daemon.DaemonService.CleanState:input_type -> daemon.CleanStateRequest - 37, // 31: daemon.DaemonService.DeleteState:input_type -> daemon.DeleteStateRequest - 39, // 32: daemon.DaemonService.SetNetworkMapPersistence:input_type -> daemon.SetNetworkMapPersistenceRequest - 2, // 33: daemon.DaemonService.Login:output_type -> daemon.LoginResponse - 4, // 34: daemon.DaemonService.WaitSSOLogin:output_type -> daemon.WaitSSOLoginResponse - 6, // 35: daemon.DaemonService.Up:output_type -> daemon.UpResponse - 8, // 36: daemon.DaemonService.Status:output_type -> daemon.StatusResponse - 10, // 37: daemon.DaemonService.Down:output_type -> daemon.DownResponse - 12, // 38: daemon.DaemonService.GetConfig:output_type -> daemon.GetConfigResponse - 21, // 39: daemon.DaemonService.ListNetworks:output_type -> daemon.ListNetworksResponse - 23, // 40: daemon.DaemonService.SelectNetworks:output_type -> daemon.SelectNetworksResponse - 23, // 41: daemon.DaemonService.DeselectNetworks:output_type -> daemon.SelectNetworksResponse - 27, // 42: daemon.DaemonService.DebugBundle:output_type -> daemon.DebugBundleResponse - 29, // 43: daemon.DaemonService.GetLogLevel:output_type -> daemon.GetLogLevelResponse - 31, // 44: daemon.DaemonService.SetLogLevel:output_type -> daemon.SetLogLevelResponse - 34, // 45: daemon.DaemonService.ListStates:output_type -> daemon.ListStatesResponse - 36, // 46: daemon.DaemonService.CleanState:output_type -> daemon.CleanStateResponse - 38, // 47: daemon.DaemonService.DeleteState:output_type -> daemon.DeleteStateResponse - 40, // 48: daemon.DaemonService.SetNetworkMapPersistence:output_type -> daemon.SetNetworkMapPersistenceResponse - 33, // [33:49] is the sub-list for method output_type - 17, // [17:33] is the sub-list for method input_type - 17, // [17:17] is the sub-list for extension type_name - 17, // [17:17] is the sub-list for extension extendee - 0, // [0:17] is the sub-list for field type_name + 41, // 16: daemon.TracePacketRequest.tcp_flags:type_name -> daemon.TCPFlags + 43, // 17: daemon.TracePacketResponse.stages:type_name -> daemon.TraceStage + 24, // 18: daemon.Network.ResolvedIPsEntry.value:type_name -> daemon.IPList + 1, // 19: daemon.DaemonService.Login:input_type -> daemon.LoginRequest + 3, // 20: daemon.DaemonService.WaitSSOLogin:input_type -> daemon.WaitSSOLoginRequest + 5, // 21: daemon.DaemonService.Up:input_type -> daemon.UpRequest + 7, // 22: daemon.DaemonService.Status:input_type -> daemon.StatusRequest + 9, // 23: daemon.DaemonService.Down:input_type -> daemon.DownRequest + 11, // 24: daemon.DaemonService.GetConfig:input_type -> daemon.GetConfigRequest + 20, // 25: daemon.DaemonService.ListNetworks:input_type -> daemon.ListNetworksRequest + 22, // 26: daemon.DaemonService.SelectNetworks:input_type -> daemon.SelectNetworksRequest + 22, // 27: daemon.DaemonService.DeselectNetworks:input_type -> daemon.SelectNetworksRequest + 26, // 28: daemon.DaemonService.DebugBundle:input_type -> daemon.DebugBundleRequest + 28, // 29: daemon.DaemonService.GetLogLevel:input_type -> daemon.GetLogLevelRequest + 30, // 30: daemon.DaemonService.SetLogLevel:input_type -> daemon.SetLogLevelRequest + 33, // 31: daemon.DaemonService.ListStates:input_type -> daemon.ListStatesRequest + 35, // 32: daemon.DaemonService.CleanState:input_type -> daemon.CleanStateRequest + 37, // 33: daemon.DaemonService.DeleteState:input_type -> daemon.DeleteStateRequest + 39, // 34: daemon.DaemonService.SetNetworkMapPersistence:input_type -> daemon.SetNetworkMapPersistenceRequest + 42, // 35: daemon.DaemonService.TracePacket:input_type -> daemon.TracePacketRequest + 2, // 36: daemon.DaemonService.Login:output_type -> daemon.LoginResponse + 4, // 37: daemon.DaemonService.WaitSSOLogin:output_type -> daemon.WaitSSOLoginResponse + 6, // 38: daemon.DaemonService.Up:output_type -> daemon.UpResponse + 8, // 39: daemon.DaemonService.Status:output_type -> daemon.StatusResponse + 10, // 40: daemon.DaemonService.Down:output_type -> daemon.DownResponse + 12, // 41: daemon.DaemonService.GetConfig:output_type -> daemon.GetConfigResponse + 21, // 42: daemon.DaemonService.ListNetworks:output_type -> daemon.ListNetworksResponse + 23, // 43: daemon.DaemonService.SelectNetworks:output_type -> daemon.SelectNetworksResponse + 23, // 44: daemon.DaemonService.DeselectNetworks:output_type -> daemon.SelectNetworksResponse + 27, // 45: daemon.DaemonService.DebugBundle:output_type -> daemon.DebugBundleResponse + 29, // 46: daemon.DaemonService.GetLogLevel:output_type -> daemon.GetLogLevelResponse + 31, // 47: daemon.DaemonService.SetLogLevel:output_type -> daemon.SetLogLevelResponse + 34, // 48: daemon.DaemonService.ListStates:output_type -> daemon.ListStatesResponse + 36, // 49: daemon.DaemonService.CleanState:output_type -> daemon.CleanStateResponse + 38, // 50: daemon.DaemonService.DeleteState:output_type -> daemon.DeleteStateResponse + 40, // 51: daemon.DaemonService.SetNetworkMapPersistence:output_type -> daemon.SetNetworkMapPersistenceResponse + 44, // 52: daemon.DaemonService.TracePacket:output_type -> daemon.TracePacketResponse + 36, // [36:53] is the sub-list for method output_type + 19, // [19:36] is the sub-list for method input_type + 19, // [19:19] is the sub-list for extension type_name + 19, // [19:19] is the sub-list for extension extendee + 0, // [0:19] is the sub-list for field type_name } func init() { file_daemon_proto_init() } @@ -3544,15 +3930,65 @@ func file_daemon_proto_init() { return nil } } + file_daemon_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TCPFlags); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_daemon_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TracePacketRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_daemon_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TraceStage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_daemon_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TracePacketResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_daemon_proto_msgTypes[0].OneofWrappers = []interface{}{} + file_daemon_proto_msgTypes[41].OneofWrappers = []interface{}{} + file_daemon_proto_msgTypes[42].OneofWrappers = []interface{}{} type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_daemon_proto_rawDesc, NumEnums: 1, - NumMessages: 41, + NumMessages: 45, NumExtensions: 0, NumServices: 1, }, diff --git a/client/proto/daemon.proto b/client/proto/daemon.proto index cddf78242dc..faababff976 100644 --- a/client/proto/daemon.proto +++ b/client/proto/daemon.proto @@ -57,6 +57,8 @@ service DaemonService { // SetNetworkMapPersistence enables or disables network map persistence rpc SetNetworkMapPersistence(SetNetworkMapPersistenceRequest) returns (SetNetworkMapPersistenceResponse) {} + + rpc TracePacket(TracePacketRequest) returns (TracePacketResponse) {} } @@ -349,3 +351,36 @@ message SetNetworkMapPersistenceRequest { } message SetNetworkMapPersistenceResponse {} + +message TCPFlags { + bool syn = 1; + bool ack = 2; + bool fin = 3; + bool rst = 4; + bool psh = 5; + bool urg = 6; +} + +message TracePacketRequest { + string source_ip = 1; + string destination_ip = 2; + string protocol = 3; + uint32 source_port = 4; + uint32 destination_port = 5; + string direction = 6; + optional TCPFlags tcp_flags = 7; + optional uint32 icmp_type = 8; + optional uint32 icmp_code = 9; +} + +message TraceStage { + string name = 1; + string message = 2; + bool allowed = 3; + optional string forwarding_details = 4; +} + +message TracePacketResponse { + repeated TraceStage stages = 1; + bool final_disposition = 2; +} diff --git a/client/proto/daemon_grpc.pb.go b/client/proto/daemon_grpc.pb.go index 39424aee938..9dcb543a80c 100644 --- a/client/proto/daemon_grpc.pb.go +++ b/client/proto/daemon_grpc.pb.go @@ -51,6 +51,7 @@ type DaemonServiceClient interface { DeleteState(ctx context.Context, in *DeleteStateRequest, opts ...grpc.CallOption) (*DeleteStateResponse, error) // SetNetworkMapPersistence enables or disables network map persistence SetNetworkMapPersistence(ctx context.Context, in *SetNetworkMapPersistenceRequest, opts ...grpc.CallOption) (*SetNetworkMapPersistenceResponse, error) + TracePacket(ctx context.Context, in *TracePacketRequest, opts ...grpc.CallOption) (*TracePacketResponse, error) } type daemonServiceClient struct { @@ -205,6 +206,15 @@ func (c *daemonServiceClient) SetNetworkMapPersistence(ctx context.Context, in * return out, nil } +func (c *daemonServiceClient) TracePacket(ctx context.Context, in *TracePacketRequest, opts ...grpc.CallOption) (*TracePacketResponse, error) { + out := new(TracePacketResponse) + err := c.cc.Invoke(ctx, "/daemon.DaemonService/TracePacket", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DaemonServiceServer is the server API for DaemonService service. // All implementations must embed UnimplementedDaemonServiceServer // for forward compatibility @@ -242,6 +252,7 @@ type DaemonServiceServer interface { DeleteState(context.Context, *DeleteStateRequest) (*DeleteStateResponse, error) // SetNetworkMapPersistence enables or disables network map persistence SetNetworkMapPersistence(context.Context, *SetNetworkMapPersistenceRequest) (*SetNetworkMapPersistenceResponse, error) + TracePacket(context.Context, *TracePacketRequest) (*TracePacketResponse, error) mustEmbedUnimplementedDaemonServiceServer() } @@ -297,6 +308,9 @@ func (UnimplementedDaemonServiceServer) DeleteState(context.Context, *DeleteStat func (UnimplementedDaemonServiceServer) SetNetworkMapPersistence(context.Context, *SetNetworkMapPersistenceRequest) (*SetNetworkMapPersistenceResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method SetNetworkMapPersistence not implemented") } +func (UnimplementedDaemonServiceServer) TracePacket(context.Context, *TracePacketRequest) (*TracePacketResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method TracePacket not implemented") +} func (UnimplementedDaemonServiceServer) mustEmbedUnimplementedDaemonServiceServer() {} // UnsafeDaemonServiceServer may be embedded to opt out of forward compatibility for this service. @@ -598,6 +612,24 @@ func _DaemonService_SetNetworkMapPersistence_Handler(srv interface{}, ctx contex return interceptor(ctx, in, info, handler) } +func _DaemonService_TracePacket_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TracePacketRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DaemonServiceServer).TracePacket(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/daemon.DaemonService/TracePacket", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DaemonServiceServer).TracePacket(ctx, req.(*TracePacketRequest)) + } + return interceptor(ctx, in, info, handler) +} + // DaemonService_ServiceDesc is the grpc.ServiceDesc for DaemonService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -669,6 +701,10 @@ var DaemonService_ServiceDesc = grpc.ServiceDesc{ MethodName: "SetNetworkMapPersistence", Handler: _DaemonService_SetNetworkMapPersistence_Handler, }, + { + MethodName: "TracePacket", + Handler: _DaemonService_TracePacket_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "daemon.proto", diff --git a/client/server/trace.go b/client/server/trace.go new file mode 100644 index 00000000000..a8004f44658 --- /dev/null +++ b/client/server/trace.go @@ -0,0 +1,119 @@ +package server + +import ( + "context" + "fmt" + "net" + + fw "github.com/netbirdio/netbird/client/firewall/manager" + "github.com/netbirdio/netbird/client/firewall/uspfilter" + "github.com/netbirdio/netbird/client/proto" +) + +type packetTracer interface { + TracePacketFromBuilder(builder *uspfilter.PacketBuilder) (*uspfilter.PacketTrace, error) +} + +func (s *Server) TracePacket(_ context.Context, req *proto.TracePacketRequest) (*proto.TracePacketResponse, error) { + s.mutex.Lock() + defer s.mutex.Unlock() + + if s.connectClient == nil || s.connectClient.Engine() == nil { + return nil, fmt.Errorf("engine not initialized") + } + + fwManager := s.connectClient.Engine().GetFirewallManager() + if fwManager == nil { + return nil, fmt.Errorf("firewall manager not initialized") + } + + tracer, ok := fwManager.(packetTracer) + if !ok { + return nil, fmt.Errorf("firewall manager does not support packet tracing") + } + + srcIP := net.ParseIP(req.GetSourceIp()) + if req.GetSourceIp() == "self" { + srcIP = s.connectClient.Engine().GetWgAddr() + } + + dstIP := net.ParseIP(req.GetDestinationIp()) + if req.GetDestinationIp() == "self" { + dstIP = s.connectClient.Engine().GetWgAddr() + } + + if srcIP == nil || dstIP == nil { + return nil, fmt.Errorf("invalid IP address") + } + + var tcpState *uspfilter.TCPState + if flags := req.GetTcpFlags(); flags != nil { + tcpState = &uspfilter.TCPState{ + SYN: flags.GetSyn(), + ACK: flags.GetAck(), + FIN: flags.GetFin(), + RST: flags.GetRst(), + PSH: flags.GetPsh(), + URG: flags.GetUrg(), + } + } + + var dir fw.RuleDirection + switch req.GetDirection() { + case "in": + dir = fw.RuleDirectionIN + case "out": + dir = fw.RuleDirectionOUT + default: + return nil, fmt.Errorf("invalid direction") + } + + var protocol fw.Protocol + switch req.GetProtocol() { + case "tcp": + protocol = fw.ProtocolTCP + case "udp": + protocol = fw.ProtocolUDP + case "icmp": + protocol = fw.ProtocolICMP + default: + return nil, fmt.Errorf("invalid protocolcol") + } + + builder := &uspfilter.PacketBuilder{ + SrcIP: srcIP, + DstIP: dstIP, + Protocol: protocol, + SrcPort: uint16(req.GetSourcePort()), + DstPort: uint16(req.GetDestinationPort()), + Direction: dir, + TCPState: tcpState, + ICMPType: uint8(req.GetIcmpType()), + ICMPCode: uint8(req.GetIcmpCode()), + } + trace, err := tracer.TracePacketFromBuilder(builder) + if err != nil { + return nil, fmt.Errorf("trace packet: %w", err) + } + + resp := &proto.TracePacketResponse{} + + for _, result := range trace.Results { + stage := &proto.TraceStage{ + Name: result.Stage.String(), + Message: result.Message, + Allowed: result.Allowed, + } + if result.ForwarderAction != nil { + details := fmt.Sprintf("%s to %s", result.ForwarderAction.Action, result.ForwarderAction.RemoteAddr) + stage.ForwardingDetails = &details + } + resp.Stages = append(resp.Stages, stage) + } + + if len(trace.Results) > 0 { + resp.FinalDisposition = trace.Results[len(trace.Results)-1].Allowed + } + + return resp, nil +} From 474fb3330584a66f2aa0707074623a2872c61fcf Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sat, 4 Jan 2025 20:52:26 +0100 Subject: [PATCH 38/59] Remove established field from udp and icmp (unused) --- client/firewall/uspfilter/conntrack/common.go | 21 ++++----------- .../uspfilter/conntrack/common_test.go | 26 ------------------- client/firewall/uspfilter/conntrack/icmp.go | 9 +++---- client/firewall/uspfilter/conntrack/tcp.go | 19 +++++++++++--- client/firewall/uspfilter/conntrack/udp.go | 9 +++---- 5 files changed, 26 insertions(+), 58 deletions(-) diff --git a/client/firewall/uspfilter/conntrack/common.go b/client/firewall/uspfilter/conntrack/common.go index e459bc75ae1..f5f5025400f 100644 --- a/client/firewall/uspfilter/conntrack/common.go +++ b/client/firewall/uspfilter/conntrack/common.go @@ -10,12 +10,11 @@ import ( // BaseConnTrack provides common fields and locking for all connection types type BaseConnTrack struct { - SourceIP net.IP - DestIP net.IP - SourcePort uint16 - DestPort uint16 - lastSeen atomic.Int64 // Unix nano for atomic access - established atomic.Bool + SourceIP net.IP + DestIP net.IP + SourcePort uint16 + DestPort uint16 + lastSeen atomic.Int64 // Unix nano for atomic access } // these small methods will be inlined by the compiler @@ -25,16 +24,6 @@ func (b *BaseConnTrack) UpdateLastSeen() { b.lastSeen.Store(time.Now().UnixNano()) } -// IsEstablished safely checks if connection is established -func (b *BaseConnTrack) IsEstablished() bool { - return b.established.Load() -} - -// SetEstablished safely sets the established state -func (b *BaseConnTrack) SetEstablished(state bool) { - b.established.Store(state) -} - // GetLastSeen safely gets the last seen timestamp func (b *BaseConnTrack) GetLastSeen() time.Time { return time.Unix(0, b.lastSeen.Load()) diff --git a/client/firewall/uspfilter/conntrack/common_test.go b/client/firewall/uspfilter/conntrack/common_test.go index 152e4e482c0..81fa64b19d7 100644 --- a/client/firewall/uspfilter/conntrack/common_test.go +++ b/client/firewall/uspfilter/conntrack/common_test.go @@ -40,32 +40,6 @@ func BenchmarkIPOperations(b *testing.B) { }) } -func BenchmarkAtomicOperations(b *testing.B) { - conn := &BaseConnTrack{} - b.Run("UpdateLastSeen", func(b *testing.B) { - for i := 0; i < b.N; i++ { - conn.UpdateLastSeen() - } - }) - - b.Run("IsEstablished", func(b *testing.B) { - for i := 0; i < b.N; i++ { - _ = conn.IsEstablished() - } - }) - - b.Run("SetEstablished", func(b *testing.B) { - for i := 0; i < b.N; i++ { - conn.SetEstablished(i%2 == 0) - } - }) - - b.Run("GetLastSeen", func(b *testing.B) { - for i := 0; i < b.N; i++ { - _ = conn.GetLastSeen() - } - }) -} // Memory pressure tests func BenchmarkMemoryPressure(b *testing.B) { diff --git a/client/firewall/uspfilter/conntrack/icmp.go b/client/firewall/uspfilter/conntrack/icmp.go index f9f5fad69e9..7796b090840 100644 --- a/client/firewall/uspfilter/conntrack/icmp.go +++ b/client/firewall/uspfilter/conntrack/icmp.go @@ -66,7 +66,6 @@ func NewICMPTracker(timeout time.Duration, logger *nblog.Logger) *ICMPTracker { // TrackOutbound records an outbound ICMP Echo Request func (t *ICMPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, id uint16, seq uint16) { key := makeICMPKey(srcIP, dstIP, id, seq) - now := time.Now().UnixNano() t.mutex.Lock() conn, exists := t.connections[key] @@ -84,15 +83,14 @@ func (t *ICMPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, id uint16, seq u ID: id, Sequence: seq, } - conn.lastSeen.Store(now) - conn.established.Store(true) + conn.UpdateLastSeen() t.connections[key] = conn t.logger.Trace("New ICMP connection %v", key) } t.mutex.Unlock() - conn.lastSeen.Store(now) + conn.UpdateLastSeen() } // IsValidInbound checks if an inbound ICMP Echo Reply matches a tracked request @@ -115,8 +113,7 @@ func (t *ICMPTracker) IsValidInbound(srcIP net.IP, dstIP net.IP, id uint16, seq return false } - return conn.IsEstablished() && - ValidateIPs(MakeIPAddr(srcIP), conn.DestIP) && + return ValidateIPs(MakeIPAddr(srcIP), conn.DestIP) && ValidateIPs(MakeIPAddr(dstIP), conn.SourceIP) && conn.ID == id && conn.Sequence == seq diff --git a/client/firewall/uspfilter/conntrack/tcp.go b/client/firewall/uspfilter/conntrack/tcp.go index 3f082d32ce8..7c12e8ad01f 100644 --- a/client/firewall/uspfilter/conntrack/tcp.go +++ b/client/firewall/uspfilter/conntrack/tcp.go @@ -5,6 +5,7 @@ package conntrack import ( "net" "sync" + "sync/atomic" "time" nblog "github.com/netbirdio/netbird/client/firewall/uspfilter/log" @@ -63,10 +64,21 @@ type TCPConnKey struct { // TCPConnTrack represents a TCP connection state type TCPConnTrack struct { BaseConnTrack - State TCPState + State TCPState + established atomic.Bool sync.RWMutex } +// IsEstablished safely checks if connection is established +func (t *TCPConnTrack) IsEstablished() bool { + return t.established.Load() +} + +// SetEstablished safely sets the established state +func (t *TCPConnTrack) SetEstablished(state bool) { + t.established.Store(state) +} + // TCPTracker manages TCP connection states type TCPTracker struct { logger *nblog.Logger @@ -97,7 +109,6 @@ func NewTCPTracker(timeout time.Duration, logger *nblog.Logger) *TCPTracker { func (t *TCPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, dstPort uint16, flags uint8) { // Create key before lock key := makeConnKey(srcIP, dstIP, srcPort, dstPort) - now := time.Now().UnixNano() t.mutex.Lock() conn, exists := t.connections[key] @@ -117,7 +128,7 @@ func (t *TCPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, d }, State: TCPStateNew, } - conn.lastSeen.Store(now) + conn.UpdateLastSeen() conn.established.Store(false) t.connections[key] = conn @@ -129,7 +140,7 @@ func (t *TCPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, d conn.Lock() t.updateState(conn, flags, true) conn.Unlock() - conn.lastSeen.Store(now) + conn.UpdateLastSeen() } // IsValidInbound checks if an inbound TCP packet matches a tracked connection diff --git a/client/firewall/uspfilter/conntrack/udp.go b/client/firewall/uspfilter/conntrack/udp.go index 5e37f5cebe8..7e030f7f2f9 100644 --- a/client/firewall/uspfilter/conntrack/udp.go +++ b/client/firewall/uspfilter/conntrack/udp.go @@ -53,7 +53,6 @@ func NewUDPTracker(timeout time.Duration, logger *nblog.Logger) *UDPTracker { // TrackOutbound records an outbound UDP connection func (t *UDPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, dstPort uint16) { key := makeConnKey(srcIP, dstIP, srcPort, dstPort) - now := time.Now().UnixNano() t.mutex.Lock() conn, exists := t.connections[key] @@ -71,15 +70,14 @@ func (t *UDPTracker) TrackOutbound(srcIP net.IP, dstIP net.IP, srcPort uint16, d DestPort: dstPort, }, } - conn.lastSeen.Store(now) - conn.established.Store(true) + conn.UpdateLastSeen() t.connections[key] = conn t.logger.Trace("New UDP connection: %v", conn) } t.mutex.Unlock() - conn.lastSeen.Store(now) + conn.UpdateLastSeen() } // IsValidInbound checks if an inbound packet matches a tracked connection @@ -98,8 +96,7 @@ func (t *UDPTracker) IsValidInbound(srcIP net.IP, dstIP net.IP, srcPort uint16, return false } - return conn.IsEstablished() && - ValidateIPs(MakeIPAddr(srcIP), conn.DestIP) && + return ValidateIPs(MakeIPAddr(srcIP), conn.DestIP) && ValidateIPs(MakeIPAddr(dstIP), conn.SourceIP) && conn.DestPort == srcPort && conn.SourcePort == dstPort From 290e6992a873fd288db45dfdd11d43486253725f Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sun, 5 Jan 2025 14:44:04 +0100 Subject: [PATCH 39/59] Demote close error levels --- client/firewall/uspfilter/conntrack/icmp.go | 2 +- client/firewall/uspfilter/conntrack/udp.go | 2 +- client/firewall/uspfilter/forwarder/tcp.go | 16 +++++----------- client/firewall/uspfilter/forwarder/udp.go | 14 +++++++------- 4 files changed, 14 insertions(+), 20 deletions(-) diff --git a/client/firewall/uspfilter/conntrack/icmp.go b/client/firewall/uspfilter/conntrack/icmp.go index 7796b090840..25cd9e87d72 100644 --- a/client/firewall/uspfilter/conntrack/icmp.go +++ b/client/firewall/uspfilter/conntrack/icmp.go @@ -139,7 +139,7 @@ func (t *ICMPTracker) cleanup() { t.ipPool.Put(conn.DestIP) delete(t.connections, key) - t.logger.Debug("ICMPTracker: removed connection %v", key) + t.logger.Debug("Removed ICMP connection %v (timeout)", key) } } } diff --git a/client/firewall/uspfilter/conntrack/udp.go b/client/firewall/uspfilter/conntrack/udp.go index 7e030f7f2f9..e73465e3195 100644 --- a/client/firewall/uspfilter/conntrack/udp.go +++ b/client/firewall/uspfilter/conntrack/udp.go @@ -124,7 +124,7 @@ func (t *UDPTracker) cleanup() { t.ipPool.Put(conn.DestIP) delete(t.connections, key) - t.logger.Trace("UDP connection timed out: %v", conn) + t.logger.Trace("Removed UDP connection %v (timeout)", conn) } } } diff --git a/client/firewall/uspfilter/forwarder/tcp.go b/client/firewall/uspfilter/forwarder/tcp.go index 3cdc4521842..6d7cf3b6a70 100644 --- a/client/firewall/uspfilter/forwarder/tcp.go +++ b/client/firewall/uspfilter/forwarder/tcp.go @@ -33,7 +33,7 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { if epErr != nil { f.logger.Error("forwarder: failed to create TCP endpoint: %v", epErr) if err := outConn.Close(); err != nil { - f.logger.Error("forwarder: outConn close error: %v", err) + f.logger.Debug("forwarder: outConn close error: %v", err) } r.Complete(true) return @@ -52,10 +52,10 @@ func (f *Forwarder) handleTCP(r *tcp.ForwarderRequest) { func (f *Forwarder) proxyTCP(id stack.TransportEndpointID, inConn *gonet.TCPConn, outConn net.Conn, ep tcpip.Endpoint) { defer func() { if err := inConn.Close(); err != nil { - f.logger.Error("forwarder: inConn close error: %v", err) + f.logger.Debug("forwarder: inConn close error: %v", err) } if err := outConn.Close(); err != nil { - f.logger.Error("forwarder: outConn close error: %v", err) + f.logger.Debug("forwarder: outConn close error: %v", err) } ep.Close() }() @@ -67,18 +67,12 @@ func (f *Forwarder) proxyTCP(id stack.TransportEndpointID, inConn *gonet.TCPConn errChan := make(chan error, 2) go func() { - n, err := io.Copy(outConn, inConn) - if err != nil && !isClosedError(err) { - f.logger.Error("inbound->outbound copy error after %d bytes: %v", n, err) - } + _, err := io.Copy(outConn, inConn) errChan <- err }() go func() { - n, err := io.Copy(inConn, outConn) - if err != nil && !isClosedError(err) { - f.logger.Error("outbound->inbound copy error after %d bytes: %v", n, err) - } + _, err := io.Copy(inConn, outConn) errChan <- err }() diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 3861607e07e..67dd216161d 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -73,10 +73,10 @@ func (f *udpForwarder) Stop() { for id, conn := range f.conns { conn.cancel() if err := conn.conn.Close(); err != nil { - f.logger.Error("forwarder: UDP conn close error for %v: %v", id, err) + f.logger.Debug("forwarder: UDP conn close error for %v: %v", id, err) } if err := conn.outConn.Close(); err != nil { - f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) + f.logger.Debug("forwarder: UDP outConn close error for %v: %v", id, err) } conn.ep.Close() @@ -107,10 +107,10 @@ func (f *udpForwarder) cleanup() { for _, idle := range idleConns { idle.conn.cancel() if err := idle.conn.conn.Close(); err != nil { - f.logger.Error("forwarder: UDP conn close error for %v: %v", idle.id, err) + f.logger.Debug("forwarder: UDP conn close error for %v: %v", idle.id, err) } if err := idle.conn.outConn.Close(); err != nil { - f.logger.Error("forwarder: UDP outConn close error for %v: %v", idle.id, err) + f.logger.Debug("forwarder: UDP outConn close error for %v: %v", idle.id, err) } idle.conn.ep.Close() @@ -154,7 +154,7 @@ func (f *Forwarder) handleUDP(r *udp.ForwarderRequest) { wq := waiter.Queue{} ep, epErr := r.CreateEndpoint(&wq) if epErr != nil { - f.logger.Error("forwarder: failed to create UDP endpoint: %v", epErr) + f.logger.Debug("forwarder: failed to create UDP endpoint: %v", epErr) if err := outConn.Close(); err != nil { f.logger.Debug("forwarder: UDP outConn close error for %v: %v", id, err) } @@ -196,10 +196,10 @@ func (f *Forwarder) proxyUDP(ctx context.Context, pConn *udpPacketConn, id stack defer func() { pConn.cancel() if err := pConn.conn.Close(); err != nil { - f.logger.Error("forwarder: UDP inConn close error for %v: %v", id, err) + f.logger.Debug("forwarder: UDP inConn close error for %v: %v", id, err) } if err := pConn.outConn.Close(); err != nil { - f.logger.Error("forwarder: UDP outConn close error for %v: %v", id, err) + f.logger.Debug("forwarder: UDP outConn close error for %v: %v", id, err) } ep.Close() From fe7a2aa2635fa91fe93701eeeb9029724ee9a486 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sun, 5 Jan 2025 22:44:04 +0100 Subject: [PATCH 40/59] Fix test --- client/firewall/uspfilter/conntrack/udp_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/client/firewall/uspfilter/conntrack/udp_test.go b/client/firewall/uspfilter/conntrack/udp_test.go index ab4e26d4fdd..fa83ee356a3 100644 --- a/client/firewall/uspfilter/conntrack/udp_test.go +++ b/client/firewall/uspfilter/conntrack/udp_test.go @@ -58,7 +58,6 @@ func TestUDPTracker_TrackOutbound(t *testing.T) { assert.True(t, conn.DestIP.Equal(dstIP)) assert.Equal(t, srcPort, conn.SourcePort) assert.Equal(t, dstPort, conn.DestPort) - assert.True(t, conn.IsEstablished()) assert.WithinDuration(t, time.Now(), conn.GetLastSeen(), 1*time.Second) } From 4a189a87ce19a64497a75ffb49d929e222320858 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 6 Jan 2025 23:47:03 +0100 Subject: [PATCH 41/59] Use MTU for udp max size --- client/firewall/uspfilter/forwarder/forwarder.go | 2 +- client/firewall/uspfilter/forwarder/udp.go | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index ffd40d09810..cb9bd0f2a3c 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -101,7 +101,7 @@ func New(iface common.IFaceMapper, logger *nblog.Logger, netstack bool) (*Forwar logger: logger, stack: s, endpoint: endpoint, - udpForwarder: newUDPForwarder(logger), + udpForwarder: newUDPForwarder(mtu, logger), ctx: ctx, cancel: cancel, netstack: netstack, diff --git a/client/firewall/uspfilter/forwarder/udp.go b/client/firewall/uspfilter/forwarder/udp.go index 67dd216161d..97e4662fd39 100644 --- a/client/firewall/uspfilter/forwarder/udp.go +++ b/client/firewall/uspfilter/forwarder/udp.go @@ -19,8 +19,7 @@ import ( ) const ( - udpTimeout = 30 * time.Second - maxPacketSize = 65535 + udpTimeout = 30 * time.Second ) type udpPacketConn struct { @@ -45,7 +44,7 @@ type idleConn struct { conn *udpPacketConn } -func newUDPForwarder(logger *nblog.Logger) *udpForwarder { +func newUDPForwarder(mtu int, logger *nblog.Logger) *udpForwarder { ctx, cancel := context.WithCancel(context.Background()) f := &udpForwarder{ logger: logger, @@ -54,7 +53,7 @@ func newUDPForwarder(logger *nblog.Logger) *udpForwarder { cancel: cancel, bufPool: sync.Pool{ New: func() any { - b := make([]byte, maxPacketSize) + b := make([]byte, mtu) return &b }, }, From 5ea39dfe8a682f67e4d32aa20677da52ca88786b Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Mon, 6 Jan 2025 23:53:12 +0100 Subject: [PATCH 42/59] Adjust limits for iOS --- client/firewall/uspfilter/forwarder/forwarder.go | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index cb9bd0f2a3c..d4984d49563 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "net" + "runtime" log "github.com/sirupsen/logrus" "gvisor.dev/gvisor/pkg/buffer" @@ -20,8 +21,10 @@ import ( ) const ( - receiveWindow = 32768 - maxInFlight = 1024 + defaultReceiveWindow = 32768 + defaultMaxInFlight = 1024 + iosReceiveWindow = 16384 + iosMaxInFlight = 256 ) type Forwarder struct { @@ -108,6 +111,13 @@ func New(iface common.IFaceMapper, logger *nblog.Logger, netstack bool) (*Forwar ip: iface.Address().IP, } + receiveWindow := defaultReceiveWindow + maxInFlight := defaultMaxInFlight + if runtime.GOOS == "ios" { + receiveWindow = iosReceiveWindow + maxInFlight = iosMaxInFlight + } + tcpForwarder := tcp.NewForwarder(s, receiveWindow, maxInFlight, f.handleTCP) s.SetTransportProtocolHandler(tcp.ProtocolNumber, tcpForwarder.HandlePacket) From daf935942ce07a7b330d397fb56b9ad10f8aba43 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 9 Jan 2025 14:08:44 +0100 Subject: [PATCH 43/59] Handle disable-server-routes flag in userspace router --- client/firewall/create.go | 4 +- client/firewall/create_linux.go | 14 ++--- client/firewall/uspfilter/uspfilter.go | 58 ++++++++++++------- .../uspfilter/uspfilter_bench_test.go | 16 ++--- .../uspfilter/uspfilter_filter_test.go | 4 +- client/firewall/uspfilter/uspfilter_test.go | 20 +++---- client/internal/acl/manager_test.go | 4 +- client/internal/dns/server_test.go | 2 +- client/internal/engine.go | 2 +- 9 files changed, 70 insertions(+), 54 deletions(-) diff --git a/client/firewall/create.go b/client/firewall/create.go index 9466f4b4d6b..37ea5ceb3fa 100644 --- a/client/firewall/create.go +++ b/client/firewall/create.go @@ -14,13 +14,13 @@ import ( ) // NewFirewall creates a firewall manager instance -func NewFirewall(iface IFaceMapper, _ *statemanager.Manager) (firewall.Manager, error) { +func NewFirewall(iface IFaceMapper, _ *statemanager.Manager, disableServerRoutes bool) (firewall.Manager, error) { if !iface.IsUserspaceBind() { return nil, fmt.Errorf("not implemented for this OS: %s", runtime.GOOS) } // use userspace packet filtering firewall - fm, err := uspfilter.Create(iface) + fm, err := uspfilter.Create(iface, disableServerRoutes) if err != nil { return nil, err } diff --git a/client/firewall/create_linux.go b/client/firewall/create_linux.go index 076d08ec27b..be1b37916bb 100644 --- a/client/firewall/create_linux.go +++ b/client/firewall/create_linux.go @@ -33,12 +33,12 @@ const SKIP_NFTABLES_ENV = "NB_SKIP_NFTABLES_CHECK" // FWType is the type for the firewall type type FWType int -func NewFirewall(iface IFaceMapper, stateManager *statemanager.Manager) (firewall.Manager, error) { +func NewFirewall(iface IFaceMapper, stateManager *statemanager.Manager, disableServerRoutes bool) (firewall.Manager, error) { // on the linux system we try to user nftables or iptables // in any case, because we need to allow netbird interface traffic // so we use AllowNetbird traffic from these firewall managers // for the userspace packet filtering firewall - fm, err := createNativeFirewall(iface, stateManager) + fm, err := createNativeFirewall(iface, stateManager, disableServerRoutes) if !iface.IsUserspaceBind() { return fm, err @@ -47,10 +47,10 @@ func NewFirewall(iface IFaceMapper, stateManager *statemanager.Manager) (firewal if err != nil { log.Warnf("failed to create native firewall: %v. Proceeding with userspace", err) } - return createUserspaceFirewall(iface, fm) + return createUserspaceFirewall(iface, fm, disableServerRoutes) } -func createNativeFirewall(iface IFaceMapper, stateManager *statemanager.Manager) (firewall.Manager, error) { +func createNativeFirewall(iface IFaceMapper, stateManager *statemanager.Manager, routes bool) (firewall.Manager, error) { fm, err := createFW(iface) if err != nil { return nil, fmt.Errorf("create firewall: %s", err) @@ -77,12 +77,12 @@ func createFW(iface IFaceMapper) (firewall.Manager, error) { } } -func createUserspaceFirewall(iface IFaceMapper, fm firewall.Manager) (firewall.Manager, error) { +func createUserspaceFirewall(iface IFaceMapper, fm firewall.Manager, disableServerRoutes bool) (firewall.Manager, error) { var errUsp error if fm != nil { - fm, errUsp = uspfilter.CreateWithNativeFirewall(iface, fm) + fm, errUsp = uspfilter.CreateWithNativeFirewall(iface, fm, disableServerRoutes) } else { - fm, errUsp = uspfilter.Create(iface) + fm, errUsp = uspfilter.Create(iface, disableServerRoutes) } if errUsp != nil { diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 0da518c9fe0..7459a9a4903 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -87,18 +87,23 @@ type decoder struct { } // Create userspace firewall manager constructor -func Create(iface common.IFaceMapper) (*Manager, error) { - return create(iface) +func Create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error) { + return create(iface, disableServerRoutes) } -func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall.Manager) (*Manager, error) { - mgr, err := create(iface) +func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableServerRoutes bool) (*Manager, error) { + mgr, err := create(iface, disableServerRoutes) if err != nil { return nil, err } mgr.nativeFirewall = nativeFirewall + if disableServerRoutes { + // skip native vs userspace router logic altogether + return mgr, nil + } + if forceUserspaceRouter, _ := strconv.ParseBool(os.Getenv(EnvForceUserspaceRouter)); forceUserspaceRouter { log.Info("userspace routing is forced") return mgr, nil @@ -125,7 +130,7 @@ func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall. return mgr, nil } -func create(iface common.IFaceMapper) (*Manager, error) { +func create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error) { disableConntrack, _ := strconv.ParseBool(os.Getenv(EnvDisableConntrack)) m := &Manager{ @@ -147,6 +152,7 @@ func create(iface common.IFaceMapper) (*Manager, error) { routeRules: make(map[string]RouteRule), wgIface: iface, localipmanager: newLocalIPManager(), + routingEnabled: false, stateful: !disableConntrack, // TODO: support changing log level from logrus logger: nblog.NewFromLogrus(log.StandardLogger()), @@ -166,23 +172,16 @@ func create(iface common.IFaceMapper) (*Manager, error) { m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout, m.logger) } - if disableRouting, _ := strconv.ParseBool(os.Getenv(EnvDisableUserspaceRouting)); disableRouting { + disableUspRouting, _ := strconv.ParseBool(os.Getenv(EnvDisableUserspaceRouting)) + if disableUspRouting || disableServerRoutes { log.Info("userspace routing is disabled") - return m, nil + } else { + m.routingEnabled = true } - intf := iface.GetWGDevice() - if intf == nil { - log.Info("forwarding not supported") - // Only supported in userspace mode as we need to inject packets back into wireguard directly - } else { - var err error - m.forwarder, err = forwarder.New(iface, m.logger, m.netstack) - if err != nil { - log.Errorf("failed to create forwarder: %v", err) - } else { - m.routingEnabled = true - } + // netstack needs the forwarder for local traffic + if m.netstack || m.routingEnabled { + m.initForwarder(iface) } if err := iface.SetFilter(m); err != nil { @@ -191,6 +190,25 @@ func create(iface common.IFaceMapper) (*Manager, error) { return m, nil } +func (m *Manager) initForwarder(iface common.IFaceMapper) { + // Only supported in userspace mode as we need to inject packets back into wireguard directly + intf := iface.GetWGDevice() + if intf == nil { + log.Info("forwarding not supported") + m.routingEnabled = false + return + } + + forwarder, err := forwarder.New(iface, m.logger, m.netstack) + if err != nil { + log.Errorf("failed to create forwarder: %v", err) + m.routingEnabled = false + return + } + + m.forwarder = forwarder +} + func (m *Manager) Init(*statemanager.Manager) error { return nil } @@ -509,8 +527,6 @@ func (m *Manager) trackICMPOutbound(d *decoder, srcIP, dstIP net.IP) { // dropFilter implements filtering logic for incoming packets. // If it returns true, the packet should be dropped. func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { - // TODO: Disable router if --disable-server-router is set - m.mutex.RLock() defer m.mutex.RUnlock() diff --git a/client/firewall/uspfilter/uspfilter_bench_test.go b/client/firewall/uspfilter/uspfilter_bench_test.go index c3974b492ff..827cc1f0cc8 100644 --- a/client/firewall/uspfilter/uspfilter_bench_test.go +++ b/client/firewall/uspfilter/uspfilter_bench_test.go @@ -158,7 +158,7 @@ func BenchmarkCoreFiltering(b *testing.B) { // Create manager and basic setup manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) defer b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -203,7 +203,7 @@ func BenchmarkStateScaling(b *testing.B) { b.Run(fmt.Sprintf("conns_%d", count), func(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -251,7 +251,7 @@ func BenchmarkEstablishmentOverhead(b *testing.B) { b.Run(sc.name, func(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -450,7 +450,7 @@ func BenchmarkRoutedNetworkReturn(b *testing.B) { b.Run(sc.name, func(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -577,7 +577,7 @@ func BenchmarkLongLivedConnections(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) defer b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -668,7 +668,7 @@ func BenchmarkShortLivedConnections(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) defer b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -787,7 +787,7 @@ func BenchmarkParallelLongLivedConnections(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) defer b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) @@ -875,7 +875,7 @@ func BenchmarkParallelShortLivedConnections(b *testing.B) { manager, _ := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) defer b.Cleanup(func() { require.NoError(b, manager.Reset(nil)) }) diff --git a/client/firewall/uspfilter/uspfilter_filter_test.go b/client/firewall/uspfilter/uspfilter_filter_test.go index 93b9471487d..73209a1529f 100644 --- a/client/firewall/uspfilter/uspfilter_filter_test.go +++ b/client/firewall/uspfilter/uspfilter_filter_test.go @@ -34,7 +34,7 @@ func TestPeerACLFiltering(t *testing.T) { }, } - manager, err := Create(ifaceMock) + manager, err := Create(ifaceMock, false) require.NoError(t, err) require.NotNil(t, manager) @@ -249,7 +249,7 @@ func setupRoutedManager(tb testing.TB, network string) *Manager { }, } - manager, err := Create(ifaceMock) + manager, err := Create(ifaceMock, false) require.NoError(tb, err) require.NotNil(tb, manager) require.True(tb, manager.routingEnabled) diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index 95f79115a8d..e13fe8062e6 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -62,7 +62,7 @@ func TestManagerCreate(t *testing.T) { SetFilterFunc: func(device.PacketFilter) error { return nil }, } - m, err := Create(ifaceMock) + m, err := Create(ifaceMock, false) if err != nil { t.Errorf("failed to create Manager: %v", err) return @@ -82,7 +82,7 @@ func TestManagerAddPeerFiltering(t *testing.T) { }, } - m, err := Create(ifaceMock) + m, err := Create(ifaceMock, false) if err != nil { t.Errorf("failed to create Manager: %v", err) return @@ -117,7 +117,7 @@ func TestManagerDeleteRule(t *testing.T) { SetFilterFunc: func(device.PacketFilter) error { return nil }, } - m, err := Create(ifaceMock) + m, err := Create(ifaceMock, false) if err != nil { t.Errorf("failed to create Manager: %v", err) return @@ -210,7 +210,7 @@ func TestAddUDPPacketHook(t *testing.T) { t.Run(tt.name, func(t *testing.T) { manager, err := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) require.NoError(t, err) manager.AddUDPPacketHook(tt.in, tt.ip, tt.dPort, tt.hook) @@ -263,7 +263,7 @@ func TestManagerReset(t *testing.T) { SetFilterFunc: func(device.PacketFilter) error { return nil }, } - m, err := Create(ifaceMock) + m, err := Create(ifaceMock, false) if err != nil { t.Errorf("failed to create Manager: %v", err) return @@ -307,7 +307,7 @@ func TestNotMatchByIP(t *testing.T) { }, } - m, err := Create(ifaceMock) + m, err := Create(ifaceMock, false) if err != nil { t.Errorf("failed to create Manager: %v", err) return @@ -376,7 +376,7 @@ func TestRemovePacketHook(t *testing.T) { } // creating manager instance - manager, err := Create(iface) + manager, err := Create(iface, false) if err != nil { t.Fatalf("Failed to create Manager: %s", err) } @@ -422,7 +422,7 @@ func TestRemovePacketHook(t *testing.T) { func TestProcessOutgoingHooks(t *testing.T) { manager, err := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) require.NoError(t, err) manager.wgNetwork = &net.IPNet{ @@ -508,7 +508,7 @@ func TestUSPFilterCreatePerformance(t *testing.T) { ifaceMock := &IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, } - manager, err := Create(ifaceMock) + manager, err := Create(ifaceMock, false) require.NoError(t, err) time.Sleep(time.Second) @@ -539,7 +539,7 @@ func TestUSPFilterCreatePerformance(t *testing.T) { func TestStatefulFirewall_UDPTracking(t *testing.T) { manager, err := Create(&IFaceMock{ SetFilterFunc: func(device.PacketFilter) error { return nil }, - }) + }, false) require.NoError(t, err) manager.wgNetwork = &net.IPNet{ diff --git a/client/internal/acl/manager_test.go b/client/internal/acl/manager_test.go index d66414af9a7..d146fef1f59 100644 --- a/client/internal/acl/manager_test.go +++ b/client/internal/acl/manager_test.go @@ -52,7 +52,7 @@ func TestDefaultManager(t *testing.T) { ifaceMock.EXPECT().GetWGDevice().Return(nil).AnyTimes() // we receive one rule from the management so for testing purposes ignore it - fw, err := firewall.NewFirewall(ifaceMock, nil) + fw, err := firewall.NewFirewall(ifaceMock, nil, false) if err != nil { t.Errorf("create firewall: %v", err) return @@ -346,7 +346,7 @@ func TestDefaultManagerEnableSSHRules(t *testing.T) { ifaceMock.EXPECT().GetWGDevice().Return(nil).AnyTimes() // we receive one rule from the management so for testing purposes ignore it - fw, err := firewall.NewFirewall(ifaceMock, nil) + fw, err := firewall.NewFirewall(ifaceMock, nil, false) if err != nil { t.Errorf("create firewall: %v", err) return diff --git a/client/internal/dns/server_test.go b/client/internal/dns/server_test.go index c166820c457..14ff1bb713e 100644 --- a/client/internal/dns/server_test.go +++ b/client/internal/dns/server_test.go @@ -849,7 +849,7 @@ func createWgInterfaceWithBind(t *testing.T) (*iface.WGIface, error) { return nil, err } - pf, err := uspfilter.Create(wgIface) + pf, err := uspfilter.Create(wgIface, false) if err != nil { t.Fatalf("failed to create uspfilter: %v", err) return nil, err diff --git a/client/internal/engine.go b/client/internal/engine.go index 8d443c8aaf0..069c748a5e3 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -464,7 +464,7 @@ func (e *Engine) createFirewall() error { } var err error - e.firewall, err = firewall.NewFirewall(e.wgInterface, e.stateManager) + e.firewall, err = firewall.NewFirewall(e.wgInterface, e.stateManager, e.config.DisableServerRoutes) if err != nil || e.firewall == nil { log.Errorf("failed creating firewall manager: %s", err) return nil From 6335ef8b486c573d18e89b89017f6e758ca60f4b Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 9 Jan 2025 14:10:33 +0100 Subject: [PATCH 44/59] Correct comment --- client/firewall/uspfilter/uspfilter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 7459a9a4903..dc3a0c24ee7 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -112,7 +112,7 @@ func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall. forceNativeRouter, _ := strconv.ParseBool(EnvForceNativeRouter) // if the OS supports routing natively, or it is explicitly requested, then we don't need to filter/route ourselves - // netstack mode won't support userspace routing as there is no interface + // netstack mode won't support native routing as there is no interface if forceNativeRouter || !netstack.IsEnabled() && mgr.nativeFirewall != nil && mgr.nativeFirewall.IsServerRouteSupported() { From 706f98c1f13ffd32a831c605738f444d6af92444 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 9 Jan 2025 17:37:12 +0100 Subject: [PATCH 45/59] Improve routing decision logic --- client/firewall/uspfilter/uspfilter.go | 114 +++++++++++++------------ 1 file changed, 61 insertions(+), 53 deletions(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index dc3a0c24ee7..454089cc3c6 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -1,6 +1,7 @@ package uspfilter import ( + "errors" "fmt" "net" "net/netip" @@ -31,11 +32,6 @@ const ( // EnvDisableUserspaceRouting disables userspace routing, to-be-routed packets will be dropped. EnvDisableUserspaceRouting = "NB_DISABLE_USERSPACE_ROUTING" - // EnvForceNativeRouter forces forwarding to the native stack (even if doesn't support routing). - // This is useful when routing/firewall setup is done manually instead of by netbird. - // This setting always disables userspace routing and filtering of routed traffic. - EnvForceNativeRouter = "NB_FORCE_NATIVE_ROUTER" - // EnvForceUserspaceRouter forces userspace routing even if native routing is available. EnvForceUserspaceRouter = "NB_FORCE_USERSPACE_ROUTER" ) @@ -88,49 +84,23 @@ type decoder struct { // Create userspace firewall manager constructor func Create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error) { - return create(iface, disableServerRoutes) + return create(iface, nil, disableServerRoutes) } func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableServerRoutes bool) (*Manager, error) { - mgr, err := create(iface, disableServerRoutes) - if err != nil { - return nil, err + if nativeFirewall == nil { + return nil, errors.New("native firewall is nil") } - mgr.nativeFirewall = nativeFirewall - - if disableServerRoutes { - // skip native vs userspace router logic altogether - return mgr, nil - } - - if forceUserspaceRouter, _ := strconv.ParseBool(os.Getenv(EnvForceUserspaceRouter)); forceUserspaceRouter { - log.Info("userspace routing is forced") - return mgr, nil - } - - forceNativeRouter, _ := strconv.ParseBool(EnvForceNativeRouter) - - // if the OS supports routing natively, or it is explicitly requested, then we don't need to filter/route ourselves - // netstack mode won't support native routing as there is no interface - if forceNativeRouter || - !netstack.IsEnabled() && mgr.nativeFirewall != nil && mgr.nativeFirewall.IsServerRouteSupported() { - - mgr.nativeRouter = true - mgr.routingEnabled = true - if mgr.forwarder != nil { - mgr.forwarder.Stop() - } - - log.Info("native routing is enabled") - return mgr, nil + mgr, err := create(iface, nativeFirewall, disableServerRoutes) + if err != nil { + return nil, err } - log.Info("userspace routing is enabled") return mgr, nil } -func create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error) { +func create(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableServerRoutes bool) (*Manager, error) { disableConntrack, _ := strconv.ParseBool(os.Getenv(EnvDisableConntrack)) m := &Manager{ @@ -147,6 +117,7 @@ func create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error return d }, }, + nativeFirewall: nativeFirewall, outgoingRules: make(map[string]RuleSet), incomingRules: make(map[string]RuleSet), routeRules: make(map[string]RouteRule), @@ -154,9 +125,8 @@ func create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error localipmanager: newLocalIPManager(), routingEnabled: false, stateful: !disableConntrack, - // TODO: support changing log level from logrus - logger: nblog.NewFromLogrus(log.StandardLogger()), - netstack: netstack.IsEnabled(), + logger: nblog.NewFromLogrus(log.StandardLogger()), + netstack: netstack.IsEnabled(), } if err := m.localipmanager.UpdateLocalIPs(iface); err != nil { @@ -172,24 +142,62 @@ func create(iface common.IFaceMapper, disableServerRoutes bool) (*Manager, error m.tcpTracker = conntrack.NewTCPTracker(conntrack.DefaultTCPTimeout, m.logger) } + m.determineRouting(iface, disableServerRoutes) + + if err := iface.SetFilter(m); err != nil { + return nil, fmt.Errorf("set filter: %w", err) + } + return m, nil +} + +func (m *Manager) determineRouting(iface common.IFaceMapper, disableServerRoutes bool) { disableUspRouting, _ := strconv.ParseBool(os.Getenv(EnvDisableUserspaceRouting)) - if disableUspRouting || disableServerRoutes { + forceUserspaceRouter, _ := strconv.ParseBool(os.Getenv(EnvForceUserspaceRouter)) + + switch { + case disableUspRouting: + m.routingEnabled = false + m.nativeRouter = false log.Info("userspace routing is disabled") - } else { + + case disableServerRoutes: + // if server routes are disabled we will let packets pass to the native stack + m.routingEnabled = true + m.nativeRouter = true + + log.Info("server routes are disabled") + + case forceUserspaceRouter: m.routingEnabled = true + m.nativeRouter = false + + log.Info("userspace routing is forced") + + case !m.netstack && m.nativeFirewall != nil && m.nativeFirewall.IsServerRouteSupported(): + // if the OS supports routing natively, then we don't need to filter/route ourselves + // netstack mode won't support native routing as there is no interface + + m.routingEnabled = true + m.nativeRouter = true + + log.Info("native routing is enabled") + + default: + m.routingEnabled = true + m.nativeRouter = false + + log.Info("userspace routing enabled by default") } // netstack needs the forwarder for local traffic - if m.netstack || m.routingEnabled { - m.initForwarder(iface) - } + if m.netstack || + m.routingEnabled && !m.nativeRouter { - if err := iface.SetFilter(m); err != nil { - return nil, fmt.Errorf("set filter: %w", err) + m.initForwarder(iface) } - return m, nil } +// initForwarder initializes the forwarder, it disables routing on errors func (m *Manager) initForwarder(iface common.IFaceMapper) { // Only supported in userspace mode as we need to inject packets back into wireguard directly intf := iface.GetWGDevice() @@ -218,7 +226,7 @@ func (m *Manager) IsServerRouteSupported() bool { } func (m *Manager) AddNatRule(pair firewall.RouterPair) error { - if m.nativeRouter { + if m.nativeRouter && m.nativeFirewall != nil { return m.nativeFirewall.AddNatRule(pair) } @@ -229,7 +237,7 @@ func (m *Manager) AddNatRule(pair firewall.RouterPair) error { // RemoveNatRule removes a routing firewall rule func (m *Manager) RemoveNatRule(pair firewall.RouterPair) error { - if m.nativeRouter { + if m.nativeRouter && m.nativeFirewall != nil { return m.nativeFirewall.RemoveNatRule(pair) } return nil @@ -313,7 +321,7 @@ func (m *Manager) AddRouteFiltering( dPort *firewall.Port, action firewall.Action, ) (firewall.Rule, error) { - if m.nativeRouter { + if m.nativeRouter && m.nativeFirewall != nil { return m.nativeFirewall.AddRouteFiltering(sources, destination, proto, sPort, dPort, action) } @@ -337,7 +345,7 @@ func (m *Manager) AddRouteFiltering( } func (m *Manager) DeleteRouteRule(rule firewall.Rule) error { - if m.nativeRouter { + if m.nativeRouter && m.nativeFirewall != nil { return m.nativeFirewall.DeleteRouteRule(rule) } From af46f259ac4ccbc2bb0b1cfc7f21a1b499576916 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Fri, 10 Jan 2025 19:33:57 +0100 Subject: [PATCH 46/59] Block wg net forwarded traffic --- .../firewall/uspfilter/forwarder/forwarder.go | 4 +-- client/firewall/uspfilter/localip.go | 2 +- client/firewall/uspfilter/uspfilter.go | 30 +++++++++++++++++++ 3 files changed, 33 insertions(+), 3 deletions(-) diff --git a/client/firewall/uspfilter/forwarder/forwarder.go b/client/firewall/uspfilter/forwarder/forwarder.go index d4984d49563..4ed152b79c9 100644 --- a/client/firewall/uspfilter/forwarder/forwarder.go +++ b/client/firewall/uspfilter/forwarder/forwarder.go @@ -64,12 +64,12 @@ func New(iface common.IFaceMapper, logger *nblog.Logger, netstack bool) (*Forwar return nil, fmt.Errorf("failed to create NIC: %v", err) } - _, bits := iface.Address().Network.Mask.Size() + ones, _ := iface.Address().Network.Mask.Size() protoAddr := tcpip.ProtocolAddress{ Protocol: ipv4.ProtocolNumber, AddressWithPrefix: tcpip.AddressWithPrefix{ Address: tcpip.AddrFromSlice(iface.Address().IP.To4()), - PrefixLen: bits, + PrefixLen: ones, }, } diff --git a/client/firewall/uspfilter/localip.go b/client/firewall/uspfilter/localip.go index 48c45c098f2..dc0c529be2e 100644 --- a/client/firewall/uspfilter/localip.go +++ b/client/firewall/uspfilter/localip.go @@ -13,7 +13,7 @@ import ( type localIPManager struct { mu sync.RWMutex - // Use bitmap for IPv4 (32 bits * 2^16 = 8KB memory) + // Use bitmap for IPv4 (32 bits * 2^16 = 256KB memory) ipv4Bitmap [1 << 16]uint32 } diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 454089cc3c6..81efc56aec8 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -144,12 +144,42 @@ func create(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableSe m.determineRouting(iface, disableServerRoutes) + if err := m.blockInvalidRouted(iface); err != nil { + log.Errorf("failed to block invalid routed traffic: %v", err) + } + if err := iface.SetFilter(m); err != nil { return nil, fmt.Errorf("set filter: %w", err) } return m, nil } +func (m *Manager) blockInvalidRouted(iface common.IFaceMapper) error { + if m.forwarder == nil { + return nil + } + wgPrefix, err := netip.ParsePrefix(iface.Address().Network.String()) + if err != nil { + return fmt.Errorf("parse wireguard network: %w", err) + } + log.Debugf("blocking invalid routed traffic for %s", wgPrefix) + + if _, err := m.AddRouteFiltering( + []netip.Prefix{netip.PrefixFrom(netip.IPv4Unspecified(), 0)}, + wgPrefix, + firewall.ProtocolALL, + nil, + nil, + firewall.ActionDrop, + ); err != nil { + return fmt.Errorf("block wg nte : %w", err) + } + + // TODO: Block networks that we're a client of + + return nil +} + func (m *Manager) determineRouting(iface common.IFaceMapper, disableServerRoutes bool) { disableUspRouting, _ := strconv.ParseBool(os.Getenv(EnvDisableUserspaceRouting)) forceUserspaceRouter, _ := strconv.ParseBool(os.Getenv(EnvForceUserspaceRouter)) From d31543cb12dfdc85494d302b0abab426923ccec2 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sat, 11 Jan 2025 01:26:26 +0100 Subject: [PATCH 47/59] Enable ssh server on freebsd --- client/cmd/ssh.go | 3 +-- client/internal/engine.go | 2 +- client/internal/engine_test.go | 2 +- client/ssh/login.go | 17 +++++++++-------- go.mod | 8 ++++---- go.sum | 8 ++++++++ 6 files changed, 24 insertions(+), 16 deletions(-) diff --git a/client/cmd/ssh.go b/client/cmd/ssh.go index 81e6c255a17..f9dbc26fc37 100644 --- a/client/cmd/ssh.go +++ b/client/cmd/ssh.go @@ -9,7 +9,6 @@ import ( "strings" "syscall" - log "github.com/sirupsen/logrus" "github.com/spf13/cobra" "github.com/netbirdio/netbird/client/internal" @@ -73,7 +72,7 @@ var sshCmd = &cobra.Command{ go func() { // blocking if err := runSSH(sshctx, host, []byte(config.SSHKey), cmd); err != nil { - log.Debug(err) + cmd.Printf("Error: %v\n", err) os.Exit(1) } cancel() diff --git a/client/internal/engine.go b/client/internal/engine.go index 7b6f269df48..2deee7a3169 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -699,7 +699,7 @@ func (e *Engine) updateSSH(sshConf *mgmProto.SSHConfig) error { } else { if sshConf.GetSshEnabled() { - if runtime.GOOS == "windows" || runtime.GOOS == "freebsd" { + if runtime.GOOS == "windows" { log.Warnf("running SSH server on %s is not supported", runtime.GOOS) return nil } diff --git a/client/internal/engine_test.go b/client/internal/engine_test.go index 1deea1cb84e..63d08e34c90 100644 --- a/client/internal/engine_test.go +++ b/client/internal/engine_test.go @@ -72,7 +72,7 @@ func TestMain(m *testing.M) { func TestEngine_SSH(t *testing.T) { // todo resolve test execution on freebsd - if runtime.GOOS == "windows" || runtime.GOOS == "freebsd" { + if runtime.GOOS == "windows" { t.Skip("skipping TestEngine_SSH") } diff --git a/client/ssh/login.go b/client/ssh/login.go index e6019578d61..578f5877548 100644 --- a/client/ssh/login.go +++ b/client/ssh/login.go @@ -2,11 +2,12 @@ package ssh import ( "fmt" - "github.com/netbirdio/netbird/util" "net" "net/netip" "os/exec" "runtime" + + "github.com/netbirdio/netbird/util" ) func getLoginCmd(user string, remoteAddr net.Addr) (loginPath string, args []string, err error) { @@ -20,17 +21,17 @@ func getLoginCmd(user string, remoteAddr net.Addr) (loginPath string, args []str return "", nil, err } - if runtime.GOOS == "linux" { - + switch runtime.GOOS { + case "linux": if util.FileExists("/etc/arch-release") && !util.FileExists("/etc/pam.d/remote") { - // detect if Arch Linux return loginPath, []string{"-f", user, "-p"}, nil } - return loginPath, []string{"-f", user, "-h", addrPort.Addr().String(), "-p"}, nil - } else if runtime.GOOS == "darwin" { + case "darwin": return loginPath, []string{"-fp", "-h", addrPort.Addr().String(), user}, nil + case "freebsd": + return loginPath, []string{"-f", user, "-h", addrPort.Addr().String(), "-p"}, nil + default: + return "", nil, fmt.Errorf("unsupported platform: %s", runtime.GOOS) } - - return "", nil, fmt.Errorf("unsupported platform") } diff --git a/go.mod b/go.mod index 9d7cb1fe425..5899fd93cbc 100644 --- a/go.mod +++ b/go.mod @@ -19,8 +19,8 @@ require ( github.com/spf13/cobra v1.7.0 github.com/spf13/pflag v1.0.5 github.com/vishvananda/netlink v1.2.1-beta.2 - golang.org/x/crypto v0.31.0 - golang.org/x/sys v0.28.0 + golang.org/x/crypto v0.32.0 + golang.org/x/sys v0.29.0 golang.zx2c4.com/wireguard v0.0.0-20230704135630-469159ecf7d1 golang.zx2c4.com/wireguard/wgctrl v0.0.0-20230429144221-925a1e7659e6 golang.zx2c4.com/wireguard/windows v0.5.3 @@ -41,7 +41,7 @@ require ( github.com/davecgh/go-spew v1.1.1 github.com/eko/gocache/v3 v3.1.1 github.com/fsnotify/fsnotify v1.7.0 - github.com/gliderlabs/ssh v0.3.4 + github.com/gliderlabs/ssh v0.3.8 github.com/godbus/dbus/v5 v5.1.0 github.com/golang/mock v1.6.0 github.com/google/go-cmp v0.6.0 @@ -94,7 +94,7 @@ require ( golang.org/x/net v0.30.0 golang.org/x/oauth2 v0.19.0 golang.org/x/sync v0.10.0 - golang.org/x/term v0.27.0 + golang.org/x/term v0.28.0 google.golang.org/api v0.177.0 gopkg.in/yaml.v3 v3.0.1 gorm.io/driver/mysql v1.5.7 diff --git a/go.sum b/go.sum index 8383475a4fe..61b01387d51 100644 --- a/go.sum +++ b/go.sum @@ -214,6 +214,8 @@ github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= github.com/gliderlabs/ssh v0.3.4 h1:+AXBtim7MTKaLVPgvE+3mhewYRawNLTd+jEEz/wExZw= github.com/gliderlabs/ssh v0.3.4/go.mod h1:ZSS+CUoKHDrqVakTfTWUlKSr9MtMFkC4UvtQKD7O914= +github.com/gliderlabs/ssh v0.3.8 h1:a4YXD1V7xMF9g5nTkdfnja3Sxy1PVDCj1Zg4Wb8vY6c= +github.com/gliderlabs/ssh v0.3.8/go.mod h1:xYoytBv1sV0aL3CavoDuJIQNURXkkfPA/wxQ1pL1fAU= github.com/go-gl/gl v0.0.0-20211210172815-726fda9656d6 h1:zDw5v7qm4yH7N8C8uWd+8Ii9rROdgWxQuGoJ9WDXxfk= github.com/go-gl/gl v0.0.0-20211210172815-726fda9656d6/go.mod h1:9YTyiznxEY1fVinfM7RvRcjRHbw2xLBJ3AAGIT0I4Nw= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= @@ -784,6 +786,8 @@ golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98y golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/crypto v0.32.0 h1:euUpcYgM8WcP71gNpTqQCn6rC2t6ULUPiOzfWaXVVfc= +golang.org/x/crypto v0.32.0/go.mod h1:ZnnJkOaASj8g0AjIduWNlq2NRxL0PlBrbKVyZ6V/Ugc= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -984,6 +988,8 @@ golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.29.0 h1:TPYlXGxvx1MGTn2GiZDhnjPA9wZzZeGKHHmKhHYvgaU= +golang.org/x/sys v0.29.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= @@ -993,6 +999,8 @@ golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= +golang.org/x/term v0.28.0 h1:/Ts8HFuMR2E6IP/jlo7QVLZHggjKQbhu/7H0LJFr3Gg= +golang.org/x/term v0.28.0/go.mod h1:Sw/lC2IAUZ92udQNf3WodGtn4k/XoLyZoh8v/8uiwek= golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 648b22aca15e6ade22dc8f3332cc5e32cf2e9fe4 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sat, 11 Jan 2025 02:10:09 +0100 Subject: [PATCH 48/59] Fix listening in netstack mode --- client/internal/engine.go | 13 +++++++++---- client/internal/engine_test.go | 1 - 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/client/internal/engine.go b/client/internal/engine.go index 2deee7a3169..4991ff9815c 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -27,6 +27,7 @@ import ( "github.com/netbirdio/netbird/client/iface" "github.com/netbirdio/netbird/client/iface/bind" "github.com/netbirdio/netbird/client/iface/device" + "github.com/netbirdio/netbird/client/iface/netstack" "github.com/netbirdio/netbird/client/internal/acl" "github.com/netbirdio/netbird/client/internal/dns" "github.com/netbirdio/netbird/client/internal/dnsfwd" @@ -705,12 +706,16 @@ func (e *Engine) updateSSH(sshConf *mgmProto.SSHConfig) error { } // start SSH server if it wasn't running if isNil(e.sshServer) { + listenAddr := fmt.Sprintf("%s:%d", e.wgInterface.Address().IP.String(), nbssh.DefaultSSHPort) + if netstack.IsEnabled() { + listenAddr = fmt.Sprintf("127.0.0.1:%d", nbssh.DefaultSSHPort) + } // nil sshServer means it has not yet been started var err error - e.sshServer, err = e.sshServerFunc(e.config.SSHKey, - fmt.Sprintf("%s:%d", e.wgInterface.Address().IP.String(), nbssh.DefaultSSHPort)) + e.sshServer, err = e.sshServerFunc(e.config.SSHKey, listenAddr) + if err != nil { - return err + return fmt.Errorf("create ssh server: %w", err) } go func() { // blocking @@ -759,7 +764,7 @@ func (e *Engine) updateConfig(conf *mgmProto.PeerConfig) error { if conf.GetSshConfig() != nil { err := e.updateSSH(conf.GetSshConfig()) if err != nil { - log.Warnf("failed handling SSH server setup %v", err) + log.Warnf("failed handling SSH server setup: %v", err) } } diff --git a/client/internal/engine_test.go b/client/internal/engine_test.go index 63d08e34c90..ca49eca09f6 100644 --- a/client/internal/engine_test.go +++ b/client/internal/engine_test.go @@ -71,7 +71,6 @@ func TestMain(m *testing.M) { } func TestEngine_SSH(t *testing.T) { - // todo resolve test execution on freebsd if runtime.GOOS == "windows" { t.Skip("skipping TestEngine_SSH") } From 8430c37dd6aa2c87ed0af11c7047e39b6fceeada Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sat, 11 Jan 2025 02:10:27 +0100 Subject: [PATCH 49/59] Fix panic if login cmd fails --- client/ssh/server.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/client/ssh/server.go b/client/ssh/server.go index a390302b704..1f2001d0f61 100644 --- a/client/ssh/server.go +++ b/client/ssh/server.go @@ -168,8 +168,12 @@ func (srv *DefaultServer) sessionHandler(session ssh.Session) { cmd := exec.Command(loginCmd, loginArgs...) go func() { <-session.Context().Done() + if cmd.Process == nil { + return + } err := cmd.Process.Kill() if err != nil { + log.Debugf("failed killing SSH process %v", err) return } }() @@ -185,7 +189,7 @@ func (srv *DefaultServer) sessionHandler(session ssh.Session) { log.Debugf("Login command: %s", cmd.String()) file, err := pty.Start(cmd) if err != nil { - log.Errorf("failed starting SSH server %v", err) + log.Errorf("failed starting SSH server: %v", err) } go func() { From 1296ecf96ef5da5bba5d0605de83589f73560eb9 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Sat, 11 Jan 2025 02:38:31 +0100 Subject: [PATCH 50/59] Tidy up go mod --- go.sum | 9 --------- 1 file changed, 9 deletions(-) diff --git a/go.sum b/go.sum index 61b01387d51..e7eab10f111 100644 --- a/go.sum +++ b/go.sum @@ -212,8 +212,6 @@ github.com/ghodss/yaml v0.0.0-20150909031657-73d445a93680/go.mod h1:4dBDuWmgqj2H github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= -github.com/gliderlabs/ssh v0.3.4 h1:+AXBtim7MTKaLVPgvE+3mhewYRawNLTd+jEEz/wExZw= -github.com/gliderlabs/ssh v0.3.4/go.mod h1:ZSS+CUoKHDrqVakTfTWUlKSr9MtMFkC4UvtQKD7O914= github.com/gliderlabs/ssh v0.3.8 h1:a4YXD1V7xMF9g5nTkdfnja3Sxy1PVDCj1Zg4Wb8vY6c= github.com/gliderlabs/ssh v0.3.8/go.mod h1:xYoytBv1sV0aL3CavoDuJIQNURXkkfPA/wxQ1pL1fAU= github.com/go-gl/gl v0.0.0-20211210172815-726fda9656d6 h1:zDw5v7qm4yH7N8C8uWd+8Ii9rROdgWxQuGoJ9WDXxfk= @@ -778,14 +776,11 @@ golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.8.0/go.mod h1:mRqEX+O9/h5TFCrQhkgjo2yKi0yYA+9ecGkdQoHrywE= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= -golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= -golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/crypto v0.32.0 h1:euUpcYgM8WcP71gNpTqQCn6rC2t6ULUPiOzfWaXVVfc= golang.org/x/crypto v0.32.0/go.mod h1:ZnnJkOaASj8g0AjIduWNlq2NRxL0PlBrbKVyZ6V/Ugc= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -986,8 +981,6 @@ golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= -golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.29.0 h1:TPYlXGxvx1MGTn2GiZDhnjPA9wZzZeGKHHmKhHYvgaU= golang.org/x/sys v0.29.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -997,8 +990,6 @@ golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= -golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= -golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/term v0.28.0 h1:/Ts8HFuMR2E6IP/jlo7QVLZHggjKQbhu/7H0LJFr3Gg= golang.org/x/term v0.28.0/go.mod h1:Sw/lC2IAUZ92udQNf3WodGtn4k/XoLyZoh8v/8uiwek= golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 8dce13113d1e939569191932d919cf9d8e6aec5f Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Tue, 14 Jan 2025 16:13:19 +0100 Subject: [PATCH 51/59] Disable local forwarding in netstack mode by default for security reasons --- client/Dockerfile-rootless | 1 + client/firewall/uspfilter/tracer.go | 6 +- client/firewall/uspfilter/uspfilter.go | 65 +++++++++++++------ .../uspfilter/uspfilter_bench_test.go | 34 +++++----- client/firewall/uspfilter/uspfilter_test.go | 8 +-- 5 files changed, 72 insertions(+), 42 deletions(-) diff --git a/client/Dockerfile-rootless b/client/Dockerfile-rootless index 62bcaf964bd..78314ba121c 100644 --- a/client/Dockerfile-rootless +++ b/client/Dockerfile-rootless @@ -9,6 +9,7 @@ USER netbird:netbird ENV NB_FOREGROUND_MODE=true ENV NB_USE_NETSTACK_MODE=true +ENV NB_ENABLE_NETSTACK_LOCAL_FORWARDING=true ENV NB_CONFIG=config.json ENV NB_DAEMON_ADDR=unix://netbird.sock ENV NB_DISABLE_DNS=true diff --git a/client/firewall/uspfilter/tracer.go b/client/firewall/uspfilter/tracer.go index 379b11ec3c6..a4c653b3b4b 100644 --- a/client/firewall/uspfilter/tracer.go +++ b/client/firewall/uspfilter/tracer.go @@ -310,8 +310,10 @@ func (m *Manager) buildConntrackStateMessage(d *decoder) string { } func (m *Manager) handleLocalDelivery(trace *PacketTrace, packetData []byte, d *decoder, srcIP, dstIP net.IP) bool { - if !m.localipmanager.IsLocalIP(dstIP) { - return false + if !m.localForwarding { + trace.AddResult(StageRouting, "Local forwarding disabled", false) + trace.AddResult(StageCompleted, "Packet dropped - local forwarding disabled", false) + return true } trace.AddResult(StageRouting, "Packet destined for local delivery", true) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 81efc56aec8..94a2f45d26d 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -34,6 +34,10 @@ const ( // EnvForceUserspaceRouter forces userspace routing even if native routing is available. EnvForceUserspaceRouter = "NB_FORCE_USERSPACE_ROUTER" + + // EnvEnableNetstackLocalForwarding enables forwarding of local traffic to the native stack when running netstack + // Leaving this on by default introduces a security risk as sockets on listening on localhost only will be accessible + EnvEnableNetstackLocalForwarding = "NB_ENABLE_NETSTACK_LOCAL_FORWARDING" ) // RuleSet is a set of rules grouped by a string key @@ -59,6 +63,8 @@ type Manager struct { stateful bool // indicates whether wireguards runs in netstack mode netstack bool + // indicates whether we forward local traffic to the native stack + localForwarding bool localipmanager *localIPManager @@ -101,7 +107,14 @@ func CreateWithNativeFirewall(iface common.IFaceMapper, nativeFirewall firewall. } func create(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableServerRoutes bool) (*Manager, error) { - disableConntrack, _ := strconv.ParseBool(os.Getenv(EnvDisableConntrack)) + disableConntrack, err := strconv.ParseBool(os.Getenv(EnvDisableConntrack)) + if err != nil { + log.Warnf("failed to parse %s: %v", EnvDisableConntrack, err) + } + enableLocalForwarding, err := strconv.ParseBool(os.Getenv(EnvEnableNetstackLocalForwarding)) + if err != nil { + log.Warnf("failed to parse %s: %v", EnvEnableNetstackLocalForwarding, err) + } m := &Manager{ decoders: sync.Pool{ @@ -127,6 +140,8 @@ func create(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableSe stateful: !disableConntrack, logger: nblog.NewFromLogrus(log.StandardLogger()), netstack: netstack.IsEnabled(), + // default true for non-netstack, for netstack only if explicitly enabled + localForwarding: !netstack.IsEnabled() || enableLocalForwarding, } if err := m.localipmanager.UpdateLocalIPs(iface); err != nil { @@ -220,7 +235,7 @@ func (m *Manager) determineRouting(iface common.IFaceMapper, disableServerRoutes } // netstack needs the forwarder for local traffic - if m.netstack || + if m.netstack && m.localForwarding || m.routingEnabled && !m.nativeRouter { m.initForwarder(iface) @@ -436,7 +451,7 @@ func (m *Manager) DropOutgoing(packetData []byte) bool { // DropIncoming filter incoming packets func (m *Manager) DropIncoming(packetData []byte) bool { - return m.dropFilter(packetData, m.incomingRules) + return m.dropFilter(packetData) } // UpdateLocalIPs updates the list of local IPs @@ -564,7 +579,7 @@ func (m *Manager) trackICMPOutbound(d *decoder, srcIP, dstIP net.IP) { // dropFilter implements filtering logic for incoming packets. // If it returns true, the packet should be dropped. -func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { +func (m *Manager) dropFilter(packetData []byte) bool { m.mutex.RLock() defer m.mutex.RUnlock() @@ -588,27 +603,37 @@ func (m *Manager) dropFilter(packetData []byte, rules map[string]RuleSet) bool { return false } - // Handle local traffic - apply peer ACLs if m.localipmanager.IsLocalIP(dstIP) { - if m.peerACLsBlock(srcIP, packetData, rules, d) { - m.logger.Trace("Dropping local packet: src=%s dst=%s rules=denied", - srcIP, dstIP) - return true - } - - // if running in netstack mode we need to pass this to the forwarder - if m.netstack { - m.handleNetstackLocalTraffic(packetData) - // don't process this packet further - return true - } - - return false + return m.handleLocalTraffic(d, srcIP, dstIP, packetData) } return m.handleRoutedTraffic(d, srcIP, dstIP, packetData) } +// handleLocalTraffic handles local traffic. +// If it returns true, the packet should be dropped. +func (m *Manager) handleLocalTraffic(d *decoder, srcIP, dstIP net.IP, packetData []byte) bool { + if !m.localForwarding { + m.logger.Trace("Dropping local packet (local forwarding disabled): src=%s dst=%s", srcIP, dstIP) + return true + } + + if m.peerACLsBlock(srcIP, packetData, m.incomingRules, d) { + m.logger.Trace("Dropping local packet (ACL denied): src=%s dst=%s", + srcIP, dstIP) + return true + } + + // if running in netstack mode we need to pass this to the forwarder + if m.netstack { + m.handleNetstackLocalTraffic(packetData) + + // don't process this packet further + return true + } + + return false +} func (m *Manager) handleNetstackLocalTraffic(packetData []byte) { if m.forwarder == nil { return @@ -619,6 +644,8 @@ func (m *Manager) handleNetstackLocalTraffic(packetData []byte) { } } +// handleRoutedTraffic handles routed traffic. +// If it returns true, the packet should be dropped. func (m *Manager) handleRoutedTraffic(d *decoder, srcIP, dstIP net.IP, packetData []byte) bool { // Drop if routing is disabled if !m.routingEnabled { diff --git a/client/firewall/uspfilter/uspfilter_bench_test.go b/client/firewall/uspfilter/uspfilter_bench_test.go index 827cc1f0cc8..684057d2431 100644 --- a/client/firewall/uspfilter/uspfilter_bench_test.go +++ b/client/firewall/uspfilter/uspfilter_bench_test.go @@ -188,7 +188,7 @@ func BenchmarkCoreFiltering(b *testing.B) { // Measure inbound packet processing b.ResetTimer() for i := 0; i < b.N; i++ { - manager.dropFilter(inbound, manager.incomingRules) + manager.dropFilter(inbound) } }) } @@ -231,7 +231,7 @@ func BenchmarkStateScaling(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - manager.dropFilter(testIn, manager.incomingRules) + manager.dropFilter(testIn) } }) } @@ -272,7 +272,7 @@ func BenchmarkEstablishmentOverhead(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - manager.dropFilter(inbound, manager.incomingRules) + manager.dropFilter(inbound) } }) } @@ -475,7 +475,7 @@ func BenchmarkRoutedNetworkReturn(b *testing.B) { manager.processOutgoingHooks(syn) // SYN-ACK synack := generateTCPPacketWithFlags(b, dstIP, srcIP, 80, 1024, uint16(conntrack.TCPSyn|conntrack.TCPAck)) - manager.dropFilter(synack, manager.incomingRules) + manager.dropFilter(synack) // ACK ack := generateTCPPacketWithFlags(b, srcIP, dstIP, 1024, 80, uint16(conntrack.TCPAck)) manager.processOutgoingHooks(ack) @@ -484,7 +484,7 @@ func BenchmarkRoutedNetworkReturn(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - manager.dropFilter(inbound, manager.incomingRules) + manager.dropFilter(inbound) } }) } @@ -621,7 +621,7 @@ func BenchmarkLongLivedConnections(b *testing.B) { // SYN-ACK synack := generateTCPPacketWithFlags(b, dstIPs[i], srcIPs[i], 80, uint16(1024+i), uint16(conntrack.TCPSyn|conntrack.TCPAck)) - manager.dropFilter(synack, manager.incomingRules) + manager.dropFilter(synack) // ACK ack := generateTCPPacketWithFlags(b, srcIPs[i], dstIPs[i], @@ -649,7 +649,7 @@ func BenchmarkLongLivedConnections(b *testing.B) { // First outbound data manager.processOutgoingHooks(outPackets[connIdx]) // Then inbound response - this is what we're actually measuring - manager.dropFilter(inPackets[connIdx], manager.incomingRules) + manager.dropFilter(inPackets[connIdx]) } }) } @@ -757,17 +757,17 @@ func BenchmarkShortLivedConnections(b *testing.B) { // Connection establishment manager.processOutgoingHooks(p.syn) - manager.dropFilter(p.synAck, manager.incomingRules) + manager.dropFilter(p.synAck) manager.processOutgoingHooks(p.ack) // Data transfer manager.processOutgoingHooks(p.request) - manager.dropFilter(p.response, manager.incomingRules) + manager.dropFilter(p.response) // Connection teardown manager.processOutgoingHooks(p.finClient) - manager.dropFilter(p.ackServer, manager.incomingRules) - manager.dropFilter(p.finServer, manager.incomingRules) + manager.dropFilter(p.ackServer) + manager.dropFilter(p.finServer) manager.processOutgoingHooks(p.ackClient) } }) @@ -828,7 +828,7 @@ func BenchmarkParallelLongLivedConnections(b *testing.B) { synack := generateTCPPacketWithFlags(b, dstIPs[i], srcIPs[i], 80, uint16(1024+i), uint16(conntrack.TCPSyn|conntrack.TCPAck)) - manager.dropFilter(synack, manager.incomingRules) + manager.dropFilter(synack) ack := generateTCPPacketWithFlags(b, srcIPs[i], dstIPs[i], uint16(1024+i), 80, uint16(conntrack.TCPAck)) @@ -855,7 +855,7 @@ func BenchmarkParallelLongLivedConnections(b *testing.B) { // Simulate bidirectional traffic manager.processOutgoingHooks(outPackets[connIdx]) - manager.dropFilter(inPackets[connIdx], manager.incomingRules) + manager.dropFilter(inPackets[connIdx]) } }) }) @@ -952,15 +952,15 @@ func BenchmarkParallelShortLivedConnections(b *testing.B) { // Full connection lifecycle manager.processOutgoingHooks(p.syn) - manager.dropFilter(p.synAck, manager.incomingRules) + manager.dropFilter(p.synAck) manager.processOutgoingHooks(p.ack) manager.processOutgoingHooks(p.request) - manager.dropFilter(p.response, manager.incomingRules) + manager.dropFilter(p.response) manager.processOutgoingHooks(p.finClient) - manager.dropFilter(p.ackServer, manager.incomingRules) - manager.dropFilter(p.finServer, manager.incomingRules) + manager.dropFilter(p.ackServer) + manager.dropFilter(p.finServer) manager.processOutgoingHooks(p.ackClient) } }) diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index e13fe8062e6..ecfc6bf960e 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -319,7 +319,7 @@ func TestNotMatchByIP(t *testing.T) { ip := net.ParseIP("0.0.0.0") proto := fw.ProtocolUDP - direction := fw.RuleDirectionOUT + direction := fw.RuleDirectionIN action := fw.ActionAccept comment := "Test rule" @@ -357,7 +357,7 @@ func TestNotMatchByIP(t *testing.T) { return } - if m.dropFilter(buf.Bytes(), m.outgoingRules) { + if m.dropFilter(buf.Bytes()) { t.Errorf("expected packet to be accepted") return } @@ -669,7 +669,7 @@ func TestStatefulFirewall_UDPTracking(t *testing.T) { for _, cp := range checkPoints { time.Sleep(cp.sleep) - drop = manager.dropFilter(inboundBuf.Bytes(), manager.incomingRules) + drop = manager.dropFilter(inboundBuf.Bytes()) require.Equal(t, cp.shouldAllow, !drop, cp.description) // If the connection should still be valid, verify it exists @@ -740,7 +740,7 @@ func TestStatefulFirewall_UDPTracking(t *testing.T) { require.NoError(t, err) // Verify the invalid packet is dropped - drop = manager.dropFilter(testBuf.Bytes(), manager.incomingRules) + drop = manager.dropFilter(testBuf.Bytes()) require.True(t, drop, tc.description) }) } From 22991b3963027dd04d823eb7517cdcba20054df3 Mon Sep 17 00:00:00 2001 From: Viktor Liu <17948409+lixmal@users.noreply.github.com> Date: Wed, 15 Jan 2025 17:51:08 +0100 Subject: [PATCH 52/59] Process drop rules first (#3167) --- client/firewall/iptables/router_linux.go | 11 +- .../firewall/nftables/manager_linux_test.go | 17 +- client/firewall/nftables/router_linux.go | 8 +- client/firewall/uspfilter/uspfilter.go | 30 ++- .../uspfilter/uspfilter_filter_test.go | 237 ++++++++++++++++++ 5 files changed, 295 insertions(+), 8 deletions(-) diff --git a/client/firewall/iptables/router_linux.go b/client/firewall/iptables/router_linux.go index d067a3e7b01..84d8cb0d92d 100644 --- a/client/firewall/iptables/router_linux.go +++ b/client/firewall/iptables/router_linux.go @@ -135,7 +135,16 @@ func (r *router) AddRouteFiltering( } rule := genRouteFilteringRuleSpec(params) - if err := r.iptablesClient.Append(tableFilter, chainRTFWD, rule...); err != nil { + // Insert DROP rules at the beginning, append ACCEPT rules at the end + var err error + if action == firewall.ActionDrop { + // after the established rule + err = r.iptablesClient.Insert(tableFilter, chainRTFWD, 2, rule...) + } else { + err = r.iptablesClient.Append(tableFilter, chainRTFWD, rule...) + } + + if err != nil { return nil, fmt.Errorf("add route rule: %v", err) } diff --git a/client/firewall/nftables/manager_linux_test.go b/client/firewall/nftables/manager_linux_test.go index 9c9637282e5..b519d55ba6d 100644 --- a/client/firewall/nftables/manager_linux_test.go +++ b/client/firewall/nftables/manager_linux_test.go @@ -107,7 +107,7 @@ func TestNftablesManager(t *testing.T) { Kind: expr.VerdictAccept, }, } - require.ElementsMatch(t, rules[0].Exprs, expectedExprs1, "expected the same expressions") + compareExprsIgnoringCounters(t, rules[0].Exprs, expectedExprs1) ipToAdd, _ := netip.AddrFromSlice(ip) add := ipToAdd.Unmap() @@ -307,3 +307,18 @@ func TestNftablesManagerCompatibilityWithIptables(t *testing.T) { stdout, stderr = runIptablesSave(t) verifyIptablesOutput(t, stdout, stderr) } + +func compareExprsIgnoringCounters(t *testing.T, got, want []expr.Any) { + t.Helper() + require.Equal(t, len(got), len(want), "expression count mismatch") + + for i := range got { + if _, isCounter := got[i].(*expr.Counter); isCounter { + _, wantIsCounter := want[i].(*expr.Counter) + require.True(t, wantIsCounter, "expected Counter at index %d", i) + continue + } + + require.Equal(t, got[i], want[i], "expression mismatch at index %d", i) + } +} diff --git a/client/firewall/nftables/router_linux.go b/client/firewall/nftables/router_linux.go index 34bc9a9bc6c..5a02e28952f 100644 --- a/client/firewall/nftables/router_linux.go +++ b/client/firewall/nftables/router_linux.go @@ -233,7 +233,13 @@ func (r *router) AddRouteFiltering( UserData: []byte(ruleKey), } - rule = r.conn.AddRule(rule) + // Insert DROP rules at the beginning, append ACCEPT rules at the end + if action == firewall.ActionDrop { + // TODO: Insert after the established rule + rule = r.conn.InsertRule(rule) + } else { + rule = r.conn.AddRule(rule) + } log.Tracef("Adding route rule %s", spew.Sdump(rule)) if err := r.conn.Flush(); err != nil { diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 99a3dcee069..fa3068642c6 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -6,7 +6,9 @@ import ( "net" "net/netip" "os" + "slices" "strconv" + "strings" "sync" "github.com/google/gopacket" @@ -43,13 +45,28 @@ const ( // RuleSet is a set of rules grouped by a string key type RuleSet map[string]PeerRule +type RouteRules []RouteRule + +func (r RouteRules) Sort() { + slices.SortStableFunc(r, func(a, b RouteRule) int { + // Deny rules come first + if a.action == firewall.ActionDrop && b.action != firewall.ActionDrop { + return -1 + } + if a.action != firewall.ActionDrop && b.action == firewall.ActionDrop { + return 1 + } + return strings.Compare(a.id, b.id) + }) +} + // Manager userspace firewall manager type Manager struct { // outgoingRules is used for hooks only outgoingRules map[string]RuleSet // incomingRules is used for filtering and hooks incomingRules map[string]RuleSet - routeRules map[string]RouteRule + routeRules RouteRules wgNetwork *net.IPNet decoders sync.Pool wgIface common.IFaceMapper @@ -135,7 +152,6 @@ func create(iface common.IFaceMapper, nativeFirewall firewall.Manager, disableSe nativeFirewall: nativeFirewall, outgoingRules: make(map[string]RuleSet), incomingRules: make(map[string]RuleSet), - routeRules: make(map[string]RouteRule), wgIface: iface, localipmanager: newLocalIPManager(), routingEnabled: false, @@ -377,7 +393,8 @@ func (m *Manager) AddRouteFiltering( action: action, } - m.routeRules[ruleID] = rule + m.routeRules = append(m.routeRules, rule) + m.routeRules.Sort() return &rule, nil } @@ -391,11 +408,14 @@ func (m *Manager) DeleteRouteRule(rule firewall.Rule) error { defer m.mutex.Unlock() ruleID := rule.GetRuleID() - if _, exists := m.routeRules[ruleID]; !exists { + idx := slices.IndexFunc(m.routeRules, func(r RouteRule) bool { + return r.id == ruleID + }) + if idx < 0 { return fmt.Errorf("route rule not found: %s", ruleID) } - delete(m.routeRules, ruleID) + m.routeRules = slices.Delete(m.routeRules, idx, idx+1) return nil } diff --git a/client/firewall/uspfilter/uspfilter_filter_test.go b/client/firewall/uspfilter/uspfilter_filter_test.go index 73209a1529f..ef1a0bed3b4 100644 --- a/client/firewall/uspfilter/uspfilter_filter_test.go +++ b/client/firewall/uspfilter/uspfilter_filter_test.go @@ -713,6 +713,56 @@ func TestRouteACLFiltering(t *testing.T) { }, shouldPass: true, }, + { + name: "Drop TCP traffic to specific destination", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{443}}, + action: fw.ActionDrop, + }, + shouldPass: false, + }, + { + name: "Drop all traffic to specific destination", + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolALL, + action: fw.ActionDrop, + }, + shouldPass: false, + }, + { + name: "Drop traffic from multiple source networks", + srcIP: "172.16.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + rule: rule{ + sources: []netip.Prefix{ + netip.MustParsePrefix("100.10.0.0/16"), + netip.MustParsePrefix("172.16.0.0/16"), + }, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionDrop, + }, + shouldPass: false, + }, } for _, tc := range testCases { @@ -742,3 +792,190 @@ func TestRouteACLFiltering(t *testing.T) { }) } } + + +func TestRouteACLOrder(t *testing.T) { + manager := setupRoutedManager(t, "10.10.0.100/16") + + type testCase struct { + name string + rules []struct { + sources []netip.Prefix + dest netip.Prefix + proto fw.Protocol + srcPort *fw.Port + dstPort *fw.Port + action fw.Action + } + packets []struct { + srcIP string + dstIP string + proto fw.Protocol + srcPort uint16 + dstPort uint16 + shouldPass bool + } + } + + testCases := []testCase{ + { + name: "Drop rules take precedence over accept", + rules: []struct { + sources []netip.Prefix + dest netip.Prefix + proto fw.Protocol + srcPort *fw.Port + dstPort *fw.Port + action fw.Action + }{ + { + // Accept rule added first + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80, 443}}, + action: fw.ActionAccept, + }, + { + // Drop rule added second but should be evaluated first + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{443}}, + action: fw.ActionDrop, + }, + }, + packets: []struct { + srcIP string + dstIP string + proto fw.Protocol + srcPort uint16 + dstPort uint16 + shouldPass bool + }{ + { + // Should be dropped by the drop rule + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + shouldPass: false, + }, + { + // Should be allowed by the accept rule (port 80 not in drop rule) + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + shouldPass: true, + }, + }, + }, + { + name: "Multiple drop rules take precedence", + rules: []struct { + sources []netip.Prefix + dest netip.Prefix + proto fw.Protocol + srcPort *fw.Port + dstPort *fw.Port + action fw.Action + }{ + { + // Accept all + sources: []netip.Prefix{netip.MustParsePrefix("0.0.0.0/0")}, + dest: netip.MustParsePrefix("0.0.0.0/0"), + proto: fw.ProtocolALL, + action: fw.ActionAccept, + }, + { + // Drop specific port + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{443}}, + action: fw.ActionDrop, + }, + { + // Drop different port + sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, + dest: netip.MustParsePrefix("192.168.1.0/24"), + proto: fw.ProtocolTCP, + dstPort: &fw.Port{Values: []int{80}}, + action: fw.ActionDrop, + }, + }, + packets: []struct { + srcIP string + dstIP string + proto fw.Protocol + srcPort uint16 + dstPort uint16 + shouldPass bool + }{ + { + // Should be dropped by first drop rule + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 443, + shouldPass: false, + }, + { + // Should be dropped by second drop rule + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 80, + shouldPass: false, + }, + { + // Should be allowed by the accept rule (different port) + srcIP: "100.10.0.1", + dstIP: "192.168.1.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + shouldPass: true, + }, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + var rules []fw.Rule + for _, r := range tc.rules { + rule, err := manager.AddRouteFiltering( + r.sources, + r.dest, + r.proto, + r.srcPort, + r.dstPort, + r.action, + ) + require.NoError(t, err) + require.NotNil(t, rule) + rules = append(rules, rule) + } + + t.Cleanup(func() { + for _, rule := range rules { + require.NoError(t, manager.DeleteRouteRule(rule)) + } + }) + + for i, p := range tc.packets { + srcIP := net.ParseIP(p.srcIP) + dstIP := net.ParseIP(p.dstIP) + + isAllowed := manager.routeACLsPass(srcIP, dstIP, p.proto, p.srcPort, p.dstPort) + require.Equal(t, p.shouldPass, isAllowed, "packet %d failed", i) + } + }) + } +} From 21a36795905e70d4e6e6bb8af6dacb52ec8996f9 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Wed, 15 Jan 2025 18:27:15 +0100 Subject: [PATCH 53/59] Fix regression --- client/firewall/uspfilter/uspfilter_filter_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/client/firewall/uspfilter/uspfilter_filter_test.go b/client/firewall/uspfilter/uspfilter_filter_test.go index ef1a0bed3b4..aceea9b39e1 100644 --- a/client/firewall/uspfilter/uspfilter_filter_test.go +++ b/client/firewall/uspfilter/uspfilter_filter_test.go @@ -149,7 +149,6 @@ func TestPeerACLFiltering(t *testing.T) { tc.ruleProto, tc.ruleSrcPort, tc.ruleDstPort, - fw.RuleDirectionIN, tc.ruleAction, "", tc.name, From 9b5c0439e90af22616e5bd45e1b70deeeed6f0e4 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 16 Jan 2025 20:28:38 +0100 Subject: [PATCH 54/59] Make debug ops a bit safer --- client/server/debug.go | 17 +++++++++++++---- client/server/trace.go | 12 ++++++++---- 2 files changed, 21 insertions(+), 8 deletions(-) diff --git a/client/server/debug.go b/client/server/debug.go index 9de80173b2f..a7ab855e815 100644 --- a/client/server/debug.go +++ b/client/server/debug.go @@ -496,12 +496,21 @@ func (s *Server) SetLogLevel(_ context.Context, req *proto.SetLogLevelRequest) ( log.SetLevel(level) - if s.connectClient != nil && - s.connectClient.Engine() != nil && - s.connectClient.Engine().GetFirewallManager() != nil { - s.connectClient.Engine().GetFirewallManager().SetLogLevel(level) + if s.connectClient == nil { + return nil, fmt.Errorf("connect client not initialized") + } + engine := s.connectClient.Engine() + if engine == nil { + return nil, fmt.Errorf("engine not initialized") } + fwManager := engine.GetFirewallManager() + if fwManager == nil { + return nil, fmt.Errorf("firewall manager not initialized") + } + + fwManager.SetLogLevel(level) + log.Infof("Log level set to %s", level.String()) return &proto.SetLogLevelResponse{}, nil diff --git a/client/server/trace.go b/client/server/trace.go index a8004f44658..66b83d8cf86 100644 --- a/client/server/trace.go +++ b/client/server/trace.go @@ -18,11 +18,15 @@ func (s *Server) TracePacket(_ context.Context, req *proto.TracePacketRequest) ( s.mutex.Lock() defer s.mutex.Unlock() - if s.connectClient == nil || s.connectClient.Engine() == nil { + if s.connectClient == nil { + return nil, fmt.Errorf("connect client not initialized") + } + engine := s.connectClient.Engine() + if engine == nil { return nil, fmt.Errorf("engine not initialized") } - fwManager := s.connectClient.Engine().GetFirewallManager() + fwManager := engine.GetFirewallManager() if fwManager == nil { return nil, fmt.Errorf("firewall manager not initialized") } @@ -34,12 +38,12 @@ func (s *Server) TracePacket(_ context.Context, req *proto.TracePacketRequest) ( srcIP := net.ParseIP(req.GetSourceIp()) if req.GetSourceIp() == "self" { - srcIP = s.connectClient.Engine().GetWgAddr() + srcIP = engine.GetWgAddr() } dstIP := net.ParseIP(req.GetDestinationIp()) if req.GetDestinationIp() == "self" { - dstIP = s.connectClient.Engine().GetWgAddr() + dstIP = engine.GetWgAddr() } if srcIP == nil || dstIP == nil { From 862d548d4d0e9699fd13afb9bbb8db7a177ee8c7 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 23 Jan 2025 16:35:18 +0100 Subject: [PATCH 55/59] Support port ranges --- client/firewall/iptables/acl_linux.go | 33 +-- .../firewall/iptables/manager_linux_test.go | 8 +- client/firewall/iptables/router_linux.go | 4 +- client/firewall/iptables/router_linux_test.go | 12 +- client/firewall/manager/port.go | 8 +- client/firewall/nftables/acl_linux.go | 33 +-- .../firewall/nftables/manager_linux_test.go | 8 +- client/firewall/nftables/router_linux.go | 6 +- client/firewall/nftables/router_linux_test.go | 12 +- client/firewall/uspfilter/rule.go | 6 +- client/firewall/uspfilter/uspfilter.go | 46 ++-- .../uspfilter/uspfilter_bench_test.go | 12 +- client/firewall/uspfilter/uspfilter_test.go | 12 +- client/internal/acl/manager.go | 13 +- client/internal/dnsfwd/manager.go | 2 +- client/internal/engine.go | 2 +- management/proto/management.pb.go | 214 +++++++++--------- management/proto/management.proto | 1 + 18 files changed, 205 insertions(+), 227 deletions(-) diff --git a/client/firewall/iptables/acl_linux.go b/client/firewall/iptables/acl_linux.go index 2e745a31e00..6c4895e05ed 100644 --- a/client/firewall/iptables/acl_linux.go +++ b/client/firewall/iptables/acl_linux.go @@ -4,7 +4,6 @@ import ( "fmt" "net" "slices" - "strconv" "github.com/coreos/go-iptables/iptables" "github.com/google/uuid" @@ -87,19 +86,10 @@ func (m *aclManager) AddPeerFiltering( action firewall.Action, ipsetName string, ) ([]firewall.Rule, error) { - var dPortVal, sPortVal string - if dPort != nil && dPort.Values != nil { - // TODO: we support only one port per rule in current implementation of ACLs - dPortVal = strconv.Itoa(dPort.Values[0]) - } - if sPort != nil && sPort.Values != nil { - sPortVal = strconv.Itoa(sPort.Values[0]) - } - chain := chainNameInputRules - ipsetName = transformIPsetName(ipsetName, sPortVal, dPortVal) - specs := filterRuleSpecs(ip, string(protocol), sPortVal, dPortVal, action, ipsetName) + ipsetName = transformIPsetName(ipsetName, sPort, dPort) + specs := filterRuleSpecs(ip, string(protocol), sPort, dPort, action, ipsetName) mangleSpecs := slices.Clone(specs) mangleSpecs = append(mangleSpecs, @@ -109,7 +99,6 @@ func (m *aclManager) AddPeerFiltering( ) specs = append(specs, "-j", actionToStr(action)) - if ipsetName != "" { if ipList, ipsetExists := m.ipsetStore.ipset(ipsetName); ipsetExists { if err := ipset.Add(ipsetName, ip.String()); err != nil { @@ -370,7 +359,7 @@ func (m *aclManager) updateState() { } // filterRuleSpecs returns the specs of a filtering rule -func filterRuleSpecs(ip net.IP, protocol, sPort, dPort string, action firewall.Action, ipsetName string) (specs []string) { +func filterRuleSpecs(ip net.IP, protocol string, sPort, dPort *firewall.Port, action firewall.Action, ipsetName string) (specs []string) { matchByIP := true // don't use IP matching if IP is ip 0.0.0.0 if ip.String() == "0.0.0.0" { @@ -387,12 +376,8 @@ func filterRuleSpecs(ip net.IP, protocol, sPort, dPort string, action firewall.A if protocol != "all" { specs = append(specs, "-p", protocol) } - if sPort != "" { - specs = append(specs, "--sport", sPort) - } - if dPort != "" { - specs = append(specs, "--dport", dPort) - } + specs = append(specs, applyPort("--sport", sPort)...) + specs = append(specs, applyPort("--dport", dPort)...) return specs } @@ -403,15 +388,15 @@ func actionToStr(action firewall.Action) string { return "DROP" } -func transformIPsetName(ipsetName string, sPort, dPort string) string { +func transformIPsetName(ipsetName string, sPort, dPort *firewall.Port) string { switch { case ipsetName == "": return "" - case sPort != "" && dPort != "": + case sPort != nil && dPort != nil: return ipsetName + "-sport-dport" - case sPort != "": + case sPort != nil: return ipsetName + "-sport" - case dPort != "": + case dPort != nil: return ipsetName + "-dport" default: return ipsetName diff --git a/client/firewall/iptables/manager_linux_test.go b/client/firewall/iptables/manager_linux_test.go index fe0bc86de2a..df849f0f6ab 100644 --- a/client/firewall/iptables/manager_linux_test.go +++ b/client/firewall/iptables/manager_linux_test.go @@ -72,7 +72,7 @@ func TestIptablesManager(t *testing.T) { t.Run("add second rule", func(t *testing.T) { ip := net.ParseIP("10.20.0.3") port := &fw.Port{ - Values: []int{8043: 8046}, + Values: []uint16{8043: 8046}, } rule2, err = manager.AddPeerFiltering(ip, "tcp", port, nil, fw.ActionAccept, "", "accept HTTPS traffic from ports range") require.NoError(t, err, "failed to add rule") @@ -95,7 +95,7 @@ func TestIptablesManager(t *testing.T) { t.Run("reset check", func(t *testing.T) { // add second rule ip := net.ParseIP("10.20.0.3") - port := &fw.Port{Values: []int{5353}} + port := &fw.Port{Values: []uint16{5353}} _, err = manager.AddPeerFiltering(ip, "udp", nil, port, fw.ActionAccept, "", "accept Fake DNS traffic") require.NoError(t, err, "failed to add rule") @@ -145,7 +145,7 @@ func TestIptablesManagerIPSet(t *testing.T) { t.Run("add second rule", func(t *testing.T) { ip := net.ParseIP("10.20.0.3") port := &fw.Port{ - Values: []int{443}, + Values: []uint16{443}, } rule2, err = manager.AddPeerFiltering(ip, "tcp", port, nil, fw.ActionAccept, "default", "accept HTTPS traffic from ports range") for _, r := range rule2 { @@ -214,7 +214,7 @@ func TestIptablesCreatePerformance(t *testing.T) { ip := net.ParseIP("10.20.0.100") start := time.Now() for i := 0; i < testMax; i++ { - port := &fw.Port{Values: []int{1000 + i}} + port := &fw.Port{Values: []uint16{uint16(1000 + i)}} _, err = manager.AddPeerFiltering(ip, "tcp", nil, port, fw.ActionAccept, "", "accept HTTP traffic") require.NoError(t, err, "failed to add rule") diff --git a/client/firewall/iptables/router_linux.go b/client/firewall/iptables/router_linux.go index d067a3e7b01..a47d3ffe698 100644 --- a/client/firewall/iptables/router_linux.go +++ b/client/firewall/iptables/router_linux.go @@ -590,10 +590,10 @@ func applyPort(flag string, port *firewall.Port) []string { if len(port.Values) > 1 { portList := make([]string, len(port.Values)) for i, p := range port.Values { - portList[i] = strconv.Itoa(p) + portList[i] = strconv.Itoa(int(p)) } return []string{"-m", "multiport", flag, strings.Join(portList, ",")} } - return []string{flag, strconv.Itoa(port.Values[0])} + return []string{flag, strconv.Itoa(int(port.Values[0]))} } diff --git a/client/firewall/iptables/router_linux_test.go b/client/firewall/iptables/router_linux_test.go index 861bf8601bf..0eb20756756 100644 --- a/client/firewall/iptables/router_linux_test.go +++ b/client/firewall/iptables/router_linux_test.go @@ -239,7 +239,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { destination: netip.MustParsePrefix("10.0.0.0/24"), proto: firewall.ProtocolTCP, sPort: nil, - dPort: &firewall.Port{Values: []int{80}}, + dPort: &firewall.Port{Values: []uint16{80}}, direction: firewall.RuleDirectionIN, action: firewall.ActionAccept, expectSet: false, @@ -252,7 +252,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { }, destination: netip.MustParsePrefix("10.0.0.0/8"), proto: firewall.ProtocolUDP, - sPort: &firewall.Port{Values: []int{1024, 2048}, IsRange: true}, + sPort: &firewall.Port{Values: []uint16{1024, 2048}, IsRange: true}, dPort: nil, direction: firewall.RuleDirectionOUT, action: firewall.ActionDrop, @@ -285,7 +285,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { sources: []netip.Prefix{netip.MustParsePrefix("172.16.0.0/12")}, destination: netip.MustParsePrefix("192.168.0.0/16"), proto: firewall.ProtocolTCP, - sPort: &firewall.Port{Values: []int{80, 443, 8080}}, + sPort: &firewall.Port{Values: []uint16{80, 443, 8080}}, dPort: nil, direction: firewall.RuleDirectionOUT, action: firewall.ActionAccept, @@ -297,7 +297,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { destination: netip.MustParsePrefix("10.0.0.0/24"), proto: firewall.ProtocolUDP, sPort: nil, - dPort: &firewall.Port{Values: []int{5000, 5100}, IsRange: true}, + dPort: &firewall.Port{Values: []uint16{5000, 5100}, IsRange: true}, direction: firewall.RuleDirectionIN, action: firewall.ActionDrop, expectSet: false, @@ -307,8 +307,8 @@ func TestRouter_AddRouteFiltering(t *testing.T) { sources: []netip.Prefix{netip.MustParsePrefix("10.0.0.0/24")}, destination: netip.MustParsePrefix("172.16.0.0/16"), proto: firewall.ProtocolTCP, - sPort: &firewall.Port{Values: []int{1024, 65535}, IsRange: true}, - dPort: &firewall.Port{Values: []int{22}}, + sPort: &firewall.Port{Values: []uint16{1024, 65535}, IsRange: true}, + dPort: &firewall.Port{Values: []uint16{22}}, direction: firewall.RuleDirectionOUT, action: firewall.ActionAccept, expectSet: false, diff --git a/client/firewall/manager/port.go b/client/firewall/manager/port.go index 9061c1e6398..df02e311777 100644 --- a/client/firewall/manager/port.go +++ b/client/firewall/manager/port.go @@ -30,7 +30,7 @@ type Port struct { IsRange bool // Values contains one value for single port, multiple values for the list of ports, or two values for the range of ports - Values []int + Values []uint16 } // String interface implementation @@ -40,7 +40,11 @@ func (p *Port) String() string { if ports != "" { ports += "," } - ports += strconv.Itoa(port) + ports += strconv.Itoa(int(port)) } + if p.IsRange { + ports = "range:" + ports + } + return ports } diff --git a/client/firewall/nftables/acl_linux.go b/client/firewall/nftables/acl_linux.go index 0d1d659afee..e79164a8000 100644 --- a/client/firewall/nftables/acl_linux.go +++ b/client/firewall/nftables/acl_linux.go @@ -327,37 +327,8 @@ func (m *AclManager) addIOFiltering( } } - if sPort != nil && len(sPort.Values) != 0 { - expressions = append(expressions, - &expr.Payload{ - DestRegister: 1, - Base: expr.PayloadBaseTransportHeader, - Offset: 0, - Len: 2, - }, - &expr.Cmp{ - Op: expr.CmpOpEq, - Register: 1, - Data: encodePort(*sPort), - }, - ) - } - - if dPort != nil && len(dPort.Values) != 0 { - expressions = append(expressions, - &expr.Payload{ - DestRegister: 1, - Base: expr.PayloadBaseTransportHeader, - Offset: 2, - Len: 2, - }, - &expr.Cmp{ - Op: expr.CmpOpEq, - Register: 1, - Data: encodePort(*dPort), - }, - ) - } + expressions = append(expressions, applyPort(sPort, true)...) + expressions = append(expressions, applyPort(dPort, false)...) mainExpressions := slices.Clone(expressions) diff --git a/client/firewall/nftables/manager_linux_test.go b/client/firewall/nftables/manager_linux_test.go index 9c9637282e5..8d693725a6d 100644 --- a/client/firewall/nftables/manager_linux_test.go +++ b/client/firewall/nftables/manager_linux_test.go @@ -74,7 +74,7 @@ func TestNftablesManager(t *testing.T) { testClient := &nftables.Conn{} - rule, err := manager.AddPeerFiltering(ip, fw.ProtocolTCP, nil, &fw.Port{Values: []int{53}}, fw.ActionDrop, "", "") + rule, err := manager.AddPeerFiltering(ip, fw.ProtocolTCP, nil, &fw.Port{Values: []uint16{53}}, fw.ActionDrop, "", "") require.NoError(t, err, "failed to add rule") err = manager.Flush() @@ -200,7 +200,7 @@ func TestNFtablesCreatePerformance(t *testing.T) { ip := net.ParseIP("10.20.0.100") start := time.Now() for i := 0; i < testMax; i++ { - port := &fw.Port{Values: []int{1000 + i}} + port := &fw.Port{Values: []uint16{uint16(1000 + i)}} _, err = manager.AddPeerFiltering(ip, "tcp", nil, port, fw.ActionAccept, "", "accept HTTP traffic") require.NoError(t, err, "failed to add rule") @@ -283,7 +283,7 @@ func TestNftablesManagerCompatibilityWithIptables(t *testing.T) { }) ip := net.ParseIP("100.96.0.1") - _, err = manager.AddPeerFiltering(ip, fw.ProtocolTCP, nil, &fw.Port{Values: []int{80}}, fw.ActionAccept, "", "test rule") + _, err = manager.AddPeerFiltering(ip, fw.ProtocolTCP, nil, &fw.Port{Values: []uint16{80}}, fw.ActionAccept, "", "test rule") require.NoError(t, err, "failed to add peer filtering rule") _, err = manager.AddRouteFiltering( @@ -291,7 +291,7 @@ func TestNftablesManagerCompatibilityWithIptables(t *testing.T) { netip.MustParsePrefix("10.1.0.0/24"), fw.ProtocolTCP, nil, - &fw.Port{Values: []int{443}}, + &fw.Port{Values: []uint16{443}}, fw.ActionAccept, ) require.NoError(t, err, "failed to add route filtering rule") diff --git a/client/firewall/nftables/router_linux.go b/client/firewall/nftables/router_linux.go index 34bc9a9bc6c..19734673b72 100644 --- a/client/firewall/nftables/router_linux.go +++ b/client/firewall/nftables/router_linux.go @@ -956,12 +956,12 @@ func applyPort(port *firewall.Port, isSource bool) []expr.Any { &expr.Cmp{ Op: expr.CmpOpGte, Register: 1, - Data: binaryutil.BigEndian.PutUint16(uint16(port.Values[0])), + Data: binaryutil.BigEndian.PutUint16(port.Values[0]), }, &expr.Cmp{ Op: expr.CmpOpLte, Register: 1, - Data: binaryutil.BigEndian.PutUint16(uint16(port.Values[1])), + Data: binaryutil.BigEndian.PutUint16(port.Values[1]), }, ) } else { @@ -980,7 +980,7 @@ func applyPort(port *firewall.Port, isSource bool) []expr.Any { exprs = append(exprs, &expr.Cmp{ Op: expr.CmpOpEq, Register: 1, - Data: binaryutil.BigEndian.PutUint16(uint16(p)), + Data: binaryutil.BigEndian.PutUint16(p), }) } } diff --git a/client/firewall/nftables/router_linux_test.go b/client/firewall/nftables/router_linux_test.go index afc4d5c3934..2a5d7168d5c 100644 --- a/client/firewall/nftables/router_linux_test.go +++ b/client/firewall/nftables/router_linux_test.go @@ -222,7 +222,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { destination: netip.MustParsePrefix("10.0.0.0/24"), proto: firewall.ProtocolTCP, sPort: nil, - dPort: &firewall.Port{Values: []int{80}}, + dPort: &firewall.Port{Values: []uint16{80}}, direction: firewall.RuleDirectionIN, action: firewall.ActionAccept, expectSet: false, @@ -235,7 +235,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { }, destination: netip.MustParsePrefix("10.0.0.0/8"), proto: firewall.ProtocolUDP, - sPort: &firewall.Port{Values: []int{1024, 2048}, IsRange: true}, + sPort: &firewall.Port{Values: []uint16{1024, 2048}, IsRange: true}, dPort: nil, direction: firewall.RuleDirectionOUT, action: firewall.ActionDrop, @@ -268,7 +268,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { sources: []netip.Prefix{netip.MustParsePrefix("172.16.0.0/12")}, destination: netip.MustParsePrefix("192.168.0.0/16"), proto: firewall.ProtocolTCP, - sPort: &firewall.Port{Values: []int{80, 443, 8080}}, + sPort: &firewall.Port{Values: []uint16{80, 443, 8080}}, dPort: nil, direction: firewall.RuleDirectionOUT, action: firewall.ActionAccept, @@ -280,7 +280,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) { destination: netip.MustParsePrefix("10.0.0.0/24"), proto: firewall.ProtocolUDP, sPort: nil, - dPort: &firewall.Port{Values: []int{5000, 5100}, IsRange: true}, + dPort: &firewall.Port{Values: []uint16{5000, 5100}, IsRange: true}, direction: firewall.RuleDirectionIN, action: firewall.ActionDrop, expectSet: false, @@ -290,8 +290,8 @@ func TestRouter_AddRouteFiltering(t *testing.T) { sources: []netip.Prefix{netip.MustParsePrefix("10.0.0.0/24")}, destination: netip.MustParsePrefix("172.16.0.0/16"), proto: firewall.ProtocolTCP, - sPort: &firewall.Port{Values: []int{1024, 65535}, IsRange: true}, - dPort: &firewall.Port{Values: []int{22}}, + sPort: &firewall.Port{Values: []uint16{1024, 65535}, IsRange: true}, + dPort: &firewall.Port{Values: []uint16{22}}, direction: firewall.RuleDirectionOUT, action: firewall.ActionAccept, expectSet: false, diff --git a/client/firewall/uspfilter/rule.go b/client/firewall/uspfilter/rule.go index 1f98ef43e57..c59d4b264ce 100644 --- a/client/firewall/uspfilter/rule.go +++ b/client/firewall/uspfilter/rule.go @@ -4,6 +4,8 @@ import ( "net" "github.com/google/gopacket" + + firewall "github.com/netbirdio/netbird/client/firewall/manager" ) // Rule to handle management of rules @@ -13,8 +15,8 @@ type Rule struct { ipLayer gopacket.LayerType matchByIP bool protoLayer gopacket.LayerType - sPort uint16 - dPort uint16 + sPort *firewall.Port + dPort *firewall.Port drop bool comment string diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index f35d971b8ab..585482059da 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -179,13 +179,8 @@ func (m *Manager) AddPeerFiltering( r.matchByIP = false } - if sPort != nil && len(sPort.Values) == 1 { - r.sPort = uint16(sPort.Values[0]) - } - - if dPort != nil && len(dPort.Values) == 1 { - r.dPort = uint16(dPort.Values[0]) - } + r.sPort = sPort + r.dPort = dPort switch proto { case firewall.ProtocolTCP: @@ -364,7 +359,7 @@ func (m *Manager) checkUDPHooks(d *decoder, dstIP net.IP, packetData []byte) boo for _, ipKey := range []string{dstIP.String(), "0.0.0.0", "::"} { if rules, exists := m.outgoingRules[ipKey]; exists { for _, rule := range rules { - if rule.udpHook != nil && (rule.dPort == 0 || rule.dPort == uint16(d.udp.DstPort)) { + if rule.udpHook != nil && portsMatch(rule.dPort, uint16(d.udp.DstPort)) { return rule.udpHook(packetData) } } @@ -484,6 +479,23 @@ func (m *Manager) applyRules(srcIP net.IP, packetData []byte, rules map[string]R return true } +func portsMatch(rulePort *firewall.Port, packetPort uint16) bool { + if rulePort == nil { + return true + } + + if rulePort.IsRange { + return packetPort >= rulePort.Values[0] && packetPort <= rulePort.Values[1] + } + + for _, p := range rulePort.Values { + if p == packetPort { + return true + } + } + return false +} + func validateRule(ip net.IP, packetData []byte, rules map[string]Rule, d *decoder) (bool, bool) { payloadLayer := d.decoded[1] for _, rule := range rules { @@ -501,13 +513,7 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]Rule, d *decode switch payloadLayer { case layers.LayerTypeTCP: - if rule.sPort == 0 && rule.dPort == 0 { - return rule.drop, true - } - if rule.sPort != 0 && rule.sPort == uint16(d.tcp.SrcPort) { - return rule.drop, true - } - if rule.dPort != 0 && rule.dPort == uint16(d.tcp.DstPort) { + if portsMatch(rule.sPort, uint16(d.tcp.SrcPort)) && portsMatch(rule.dPort, uint16(d.tcp.DstPort)) { return rule.drop, true } case layers.LayerTypeUDP: @@ -517,13 +523,7 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]Rule, d *decode return rule.udpHook(packetData), true } - if rule.sPort == 0 && rule.dPort == 0 { - return rule.drop, true - } - if rule.sPort != 0 && rule.sPort == uint16(d.udp.SrcPort) { - return rule.drop, true - } - if rule.dPort != 0 && rule.dPort == uint16(d.udp.DstPort) { + if portsMatch(rule.sPort, uint16(d.tcp.SrcPort)) && portsMatch(rule.dPort, uint16(d.tcp.DstPort)) { return rule.drop, true } case layers.LayerTypeICMPv4, layers.LayerTypeICMPv6: @@ -548,7 +548,7 @@ func (m *Manager) AddUDPPacketHook( id: uuid.New().String(), ip: ip, protoLayer: layers.LayerTypeUDP, - dPort: dPort, + dPort: &firewall.Port{Values: []uint16{dPort}}, ipLayer: layers.LayerTypeIPv6, comment: fmt.Sprintf("UDP Hook direction: %v, ip:%v, dport:%d", in, ip, dPort), udpHook: hook, diff --git a/client/firewall/uspfilter/uspfilter_bench_test.go b/client/firewall/uspfilter/uspfilter_bench_test.go index 4a210bf47b9..46bc4439d83 100644 --- a/client/firewall/uspfilter/uspfilter_bench_test.go +++ b/client/firewall/uspfilter/uspfilter_bench_test.go @@ -112,8 +112,8 @@ func BenchmarkCoreFiltering(b *testing.B) { for i := 0; i < 1000; i++ { // Simulate realistic ruleset size ip := generateRandomIPs(1)[0] _, err := m.AddPeerFiltering(ip, fw.ProtocolTCP, - &fw.Port{Values: []int{1024 + i}}, - &fw.Port{Values: []int{80}}, + &fw.Port{Values: []uint16{uint16(1024 + i)}}, + &fw.Port{Values: []uint16{80}}, fw.ActionAccept, "", "explicit return") require.NoError(b, err) } @@ -588,7 +588,7 @@ func BenchmarkLongLivedConnections(b *testing.B) { if sc.rules { // Single rule to allow all return traffic from port 80 _, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP, - &fw.Port{Values: []int{80}}, + &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "", "return traffic") require.NoError(b, err) @@ -679,7 +679,7 @@ func BenchmarkShortLivedConnections(b *testing.B) { if sc.rules { // Single rule to allow all return traffic from port 80 _, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP, - &fw.Port{Values: []int{80}}, + &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "", "return traffic") require.NoError(b, err) @@ -797,7 +797,7 @@ func BenchmarkParallelLongLivedConnections(b *testing.B) { // Setup initial state based on scenario if sc.rules { _, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP, - &fw.Port{Values: []int{80}}, + &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "", "return traffic") require.NoError(b, err) @@ -884,7 +884,7 @@ func BenchmarkParallelShortLivedConnections(b *testing.B) { if sc.rules { _, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP, - &fw.Port{Values: []int{80}}, + &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "", "return traffic") require.NoError(b, err) diff --git a/client/firewall/uspfilter/uspfilter_test.go b/client/firewall/uspfilter/uspfilter_test.go index 7e87443aa87..9d795de691f 100644 --- a/client/firewall/uspfilter/uspfilter_test.go +++ b/client/firewall/uspfilter/uspfilter_test.go @@ -69,7 +69,7 @@ func TestManagerAddPeerFiltering(t *testing.T) { ip := net.ParseIP("192.168.1.1") proto := fw.ProtocolTCP - port := &fw.Port{Values: []int{80}} + port := &fw.Port{Values: []uint16{80}} action := fw.ActionDrop comment := "Test rule" @@ -103,7 +103,7 @@ func TestManagerDeleteRule(t *testing.T) { ip := net.ParseIP("192.168.1.1") proto := fw.ProtocolTCP - port := &fw.Port{Values: []int{80}} + port := &fw.Port{Values: []uint16{80}} action := fw.ActionDrop comment := "Test rule 2" @@ -194,8 +194,8 @@ func TestAddUDPPacketHook(t *testing.T) { t.Errorf("expected ip %s, got %s", tt.ip, addedRule.ip) return } - if tt.dPort != addedRule.dPort { - t.Errorf("expected dPort %d, got %d", tt.dPort, addedRule.dPort) + if tt.dPort != addedRule.dPort.Values[0] { + t.Errorf("expected dPort %d, got %d", tt.dPort, addedRule.dPort.Values[0]) return } if layers.LayerTypeUDP != addedRule.protoLayer { @@ -223,7 +223,7 @@ func TestManagerReset(t *testing.T) { ip := net.ParseIP("192.168.1.1") proto := fw.ProtocolTCP - port := &fw.Port{Values: []int{80}} + port := &fw.Port{Values: []uint16{80}} action := fw.ActionDrop comment := "Test rule" @@ -463,7 +463,7 @@ func TestUSPFilterCreatePerformance(t *testing.T) { ip := net.ParseIP("10.20.0.100") start := time.Now() for i := 0; i < testMax; i++ { - port := &fw.Port{Values: []int{1000 + i}} + port := &fw.Port{Values: []uint16{uint16(1000 + i)}} _, err = manager.AddPeerFiltering(ip, "tcp", nil, port, fw.ActionAccept, "", "accept HTTP traffic") require.NoError(t, err, "failed to add rule") diff --git a/client/internal/acl/manager.go b/client/internal/acl/manager.go index 0ade5d7ce32..9ec0bb031de 100644 --- a/client/internal/acl/manager.go +++ b/client/internal/acl/manager.go @@ -268,13 +268,16 @@ func (d *DefaultManager) protoRuleToFirewallRule( } var port *firewall.Port - if r.Port != "" { + if r.PortInfo != nil { + port = convertPortInfo(r.PortInfo) + } else if r.Port != "" { + // old version of management, single port value, err := strconv.Atoi(r.Port) if err != nil { - return "", nil, fmt.Errorf("invalid port, skipping firewall rule") + return "", nil, fmt.Errorf("invalid port: %w", err) } port = &firewall.Port{ - Values: []int{value}, + Values: []uint16{uint16(value)}, } } @@ -539,14 +542,14 @@ func convertPortInfo(portInfo *mgmProto.PortInfo) *firewall.Port { if portInfo.GetPort() != 0 { return &firewall.Port{ - Values: []int{int(portInfo.GetPort())}, + Values: []uint16{uint16(int(portInfo.GetPort()))}, } } if portInfo.GetRange() != nil { return &firewall.Port{ IsRange: true, - Values: []int{int(portInfo.GetRange().Start), int(portInfo.GetRange().End)}, + Values: []uint16{uint16(portInfo.GetRange().Start), uint16(portInfo.GetRange().End)}, } } diff --git a/client/internal/dnsfwd/manager.go b/client/internal/dnsfwd/manager.go index 968f2d39853..5d3036ddea9 100644 --- a/client/internal/dnsfwd/manager.go +++ b/client/internal/dnsfwd/manager.go @@ -81,7 +81,7 @@ func (m *Manager) Stop(ctx context.Context) error { func (h *Manager) allowDNSFirewall() error { dport := &firewall.Port{ IsRange: false, - Values: []int{ListenPort}, + Values: []uint16{ListenPort}, } if h.firewall == nil { diff --git a/client/internal/engine.go b/client/internal/engine.go index b3689c91153..43749fbe552 100644 --- a/client/internal/engine.go +++ b/client/internal/engine.go @@ -495,7 +495,7 @@ func (e *Engine) initFirewall() error { } rosenpassPort := e.rpManager.GetAddress().Port - port := manager.Port{Values: []int{rosenpassPort}} + port := manager.Port{Values: []uint16{uint16(rosenpassPort)}} // this rule is static and will be torn down on engine down by the firewall manager if _, err := e.firewall.AddPeerFiltering( diff --git a/management/proto/management.pb.go b/management/proto/management.pb.go index 7846c286d92..2fc87a0a29e 100644 --- a/management/proto/management.pb.go +++ b/management/proto/management.pb.go @@ -2624,6 +2624,7 @@ type FirewallRule struct { Action RuleAction `protobuf:"varint,3,opt,name=Action,proto3,enum=management.RuleAction" json:"Action,omitempty"` Protocol RuleProtocol `protobuf:"varint,4,opt,name=Protocol,proto3,enum=management.RuleProtocol" json:"Protocol,omitempty"` Port string `protobuf:"bytes,5,opt,name=Port,proto3" json:"Port,omitempty"` + PortInfo *PortInfo `protobuf:"bytes,6,opt,name=portInfo,proto3" json:"portInfo,omitempty"` } func (x *FirewallRule) Reset() { @@ -2693,6 +2694,13 @@ func (x *FirewallRule) GetPort() string { return "" } +func (x *FirewallRule) GetPortInfo() *PortInfo { + if x != nil { + return x.PortInfo + } + return nil +} + type NetworkAddress struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3397,7 +3405,7 @@ var file_management_proto_rawDesc = []byte{ 0x52, 0x02, 0x49, 0x50, 0x12, 0x16, 0x0a, 0x06, 0x4e, 0x53, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x4e, 0x53, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, - 0x22, 0xd9, 0x01, 0x0a, 0x0c, 0x46, 0x69, 0x72, 0x65, 0x77, 0x61, 0x6c, 0x6c, 0x52, 0x75, 0x6c, + 0x22, 0x8b, 0x02, 0x0a, 0x0c, 0x46, 0x69, 0x72, 0x65, 0x77, 0x61, 0x6c, 0x6c, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x50, 0x12, 0x37, 0x0a, 0x09, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x6d, @@ -3410,87 +3418,90 @@ var file_management_proto_rawDesc = []byte{ 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x52, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x22, 0x38, 0x0a, 0x0e, - 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, - 0x0a, 0x05, 0x6e, 0x65, 0x74, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, - 0x65, 0x74, 0x49, 0x50, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x61, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6d, 0x61, 0x63, 0x22, 0x1e, 0x0a, 0x06, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, - 0x12, 0x14, 0x0a, 0x05, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x05, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x22, 0x96, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x49, - 0x6e, 0x66, 0x6f, 0x12, 0x14, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0d, 0x48, 0x00, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x32, 0x0a, 0x05, 0x72, 0x61, 0x6e, - 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x1a, 0x2f, 0x0a, - 0x05, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, - 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x42, 0x0f, - 0x0a, 0x0d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, - 0xd1, 0x02, 0x0a, 0x11, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x46, 0x69, 0x72, 0x65, 0x77, 0x61, 0x6c, - 0x6c, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x06, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x52, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x0a, 0x08, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x52, 0x75, 0x6c, 0x65, 0x50, - 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, - 0x6c, 0x12, 0x30, 0x0a, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, - 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x49, - 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x69, 0x73, 0x44, 0x79, 0x6e, 0x61, 0x6d, 0x69, 0x63, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x44, 0x79, 0x6e, 0x61, 0x6d, 0x69, - 0x63, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x07, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, - 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x2a, 0x4c, 0x0a, 0x0c, 0x52, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, - 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x54, 0x43, 0x50, - 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x55, 0x44, 0x50, 0x10, 0x03, 0x12, 0x08, 0x0a, 0x04, 0x49, - 0x43, 0x4d, 0x50, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x55, 0x53, 0x54, 0x4f, 0x4d, 0x10, - 0x05, 0x2a, 0x20, 0x0a, 0x0d, 0x52, 0x75, 0x6c, 0x65, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x49, 0x4e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4f, 0x55, - 0x54, 0x10, 0x01, 0x2a, 0x22, 0x0a, 0x0a, 0x52, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x43, 0x45, 0x50, 0x54, 0x10, 0x00, 0x12, 0x08, 0x0a, - 0x04, 0x44, 0x52, 0x4f, 0x50, 0x10, 0x01, 0x32, 0x90, 0x04, 0x0a, 0x11, 0x4d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x45, 0x0a, - 0x05, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x08, + 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, + 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x50, 0x6f, 0x72, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x38, + 0x0a, 0x0e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, + 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x65, 0x74, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x6e, 0x65, 0x74, 0x49, 0x50, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x61, 0x63, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6d, 0x61, 0x63, 0x22, 0x1e, 0x0a, 0x06, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x22, 0x96, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x72, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x14, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0d, 0x48, 0x00, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x32, 0x0a, 0x05, 0x72, + 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x1a, + 0x2f, 0x0a, 0x05, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, + 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x65, 0x6e, 0x64, + 0x42, 0x0f, 0x0a, 0x0d, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0xd1, 0x02, 0x0a, 0x11, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x46, 0x69, 0x72, 0x65, 0x77, + 0x61, 0x6c, 0x6c, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x06, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x52, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x0a, + 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x18, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x52, 0x75, 0x6c, + 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x12, 0x30, 0x0a, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x6f, 0x72, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x69, 0x73, 0x44, 0x79, 0x6e, 0x61, 0x6d, + 0x69, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x44, 0x79, 0x6e, 0x61, + 0x6d, 0x69, 0x63, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x07, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x26, 0x0a, + 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x50, 0x72, 0x6f, + 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2a, 0x4c, 0x0a, 0x0c, 0x52, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, + 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, + 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x54, + 0x43, 0x50, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x55, 0x44, 0x50, 0x10, 0x03, 0x12, 0x08, 0x0a, + 0x04, 0x49, 0x43, 0x4d, 0x50, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x55, 0x53, 0x54, 0x4f, + 0x4d, 0x10, 0x05, 0x2a, 0x20, 0x0a, 0x0d, 0x52, 0x75, 0x6c, 0x65, 0x44, 0x69, 0x72, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x49, 0x4e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, + 0x4f, 0x55, 0x54, 0x10, 0x01, 0x2a, 0x22, 0x0a, 0x0a, 0x52, 0x75, 0x6c, 0x65, 0x41, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x43, 0x45, 0x50, 0x54, 0x10, 0x00, 0x12, + 0x08, 0x0a, 0x04, 0x44, 0x52, 0x4f, 0x50, 0x10, 0x01, 0x32, 0x90, 0x04, 0x0a, 0x11, 0x4d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, + 0x45, 0x0a, 0x05, 0x4c, 0x6f, 0x67, 0x69, 0x6e, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, - 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x22, 0x00, 0x12, 0x46, 0x0a, 0x04, 0x53, 0x79, 0x6e, 0x63, 0x12, 0x1c, 0x2e, 0x6d, + 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x12, 0x46, 0x0a, 0x04, 0x53, 0x79, 0x6e, 0x63, 0x12, 0x1c, + 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, + 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, - 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, - 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x42, 0x0a, 0x0c, - 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x79, 0x12, 0x11, 0x2e, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, - 0x1d, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x53, 0x65, 0x72, - 0x76, 0x65, 0x72, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x33, 0x0a, 0x09, 0x69, 0x73, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x12, 0x11, 0x2e, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x1a, 0x11, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6d, - 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x44, 0x65, 0x76, 0x69, - 0x63, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, - 0x6c, 0x6f, 0x77, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x42, + 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x79, 0x12, 0x11, + 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, + 0x79, 0x1a, 0x1d, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x53, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x4b, 0x65, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x33, 0x0a, 0x09, 0x69, 0x73, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x12, + 0x11, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x1a, 0x11, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x44, 0x65, + 0x76, 0x69, 0x63, 0x65, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, - 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, - 0x00, 0x12, 0x58, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x50, 0x4b, 0x43, 0x45, 0x41, 0x75, 0x74, 0x68, - 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x1c, 0x2e, + 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x50, 0x4b, 0x43, 0x45, 0x41, 0x75, + 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, + 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, + 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, - 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1c, 0x2e, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, - 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x12, 0x3d, 0x0a, 0x08, 0x53, - 0x79, 0x6e, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x11, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, - 0x6e, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x42, 0x08, 0x5a, 0x06, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x70, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x12, 0x3d, 0x0a, + 0x08, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1c, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x11, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x42, 0x08, 0x5a, 0x06, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -3596,29 +3607,30 @@ var file_management_proto_depIdxs = []int32{ 1, // 39: management.FirewallRule.Direction:type_name -> management.RuleDirection 2, // 40: management.FirewallRule.Action:type_name -> management.RuleAction 0, // 41: management.FirewallRule.Protocol:type_name -> management.RuleProtocol - 42, // 42: management.PortInfo.range:type_name -> management.PortInfo.Range - 2, // 43: management.RouteFirewallRule.action:type_name -> management.RuleAction - 0, // 44: management.RouteFirewallRule.protocol:type_name -> management.RuleProtocol - 40, // 45: management.RouteFirewallRule.portInfo:type_name -> management.PortInfo - 5, // 46: management.ManagementService.Login:input_type -> management.EncryptedMessage - 5, // 47: management.ManagementService.Sync:input_type -> management.EncryptedMessage - 17, // 48: management.ManagementService.GetServerKey:input_type -> management.Empty - 17, // 49: management.ManagementService.isHealthy:input_type -> management.Empty - 5, // 50: management.ManagementService.GetDeviceAuthorizationFlow:input_type -> management.EncryptedMessage - 5, // 51: management.ManagementService.GetPKCEAuthorizationFlow:input_type -> management.EncryptedMessage - 5, // 52: management.ManagementService.SyncMeta:input_type -> management.EncryptedMessage - 5, // 53: management.ManagementService.Login:output_type -> management.EncryptedMessage - 5, // 54: management.ManagementService.Sync:output_type -> management.EncryptedMessage - 16, // 55: management.ManagementService.GetServerKey:output_type -> management.ServerKeyResponse - 17, // 56: management.ManagementService.isHealthy:output_type -> management.Empty - 5, // 57: management.ManagementService.GetDeviceAuthorizationFlow:output_type -> management.EncryptedMessage - 5, // 58: management.ManagementService.GetPKCEAuthorizationFlow:output_type -> management.EncryptedMessage - 17, // 59: management.ManagementService.SyncMeta:output_type -> management.Empty - 53, // [53:60] is the sub-list for method output_type - 46, // [46:53] is the sub-list for method input_type - 46, // [46:46] is the sub-list for extension type_name - 46, // [46:46] is the sub-list for extension extendee - 0, // [0:46] is the sub-list for field type_name + 40, // 42: management.FirewallRule.portInfo:type_name -> management.PortInfo + 42, // 43: management.PortInfo.range:type_name -> management.PortInfo.Range + 2, // 44: management.RouteFirewallRule.action:type_name -> management.RuleAction + 0, // 45: management.RouteFirewallRule.protocol:type_name -> management.RuleProtocol + 40, // 46: management.RouteFirewallRule.portInfo:type_name -> management.PortInfo + 5, // 47: management.ManagementService.Login:input_type -> management.EncryptedMessage + 5, // 48: management.ManagementService.Sync:input_type -> management.EncryptedMessage + 17, // 49: management.ManagementService.GetServerKey:input_type -> management.Empty + 17, // 50: management.ManagementService.isHealthy:input_type -> management.Empty + 5, // 51: management.ManagementService.GetDeviceAuthorizationFlow:input_type -> management.EncryptedMessage + 5, // 52: management.ManagementService.GetPKCEAuthorizationFlow:input_type -> management.EncryptedMessage + 5, // 53: management.ManagementService.SyncMeta:input_type -> management.EncryptedMessage + 5, // 54: management.ManagementService.Login:output_type -> management.EncryptedMessage + 5, // 55: management.ManagementService.Sync:output_type -> management.EncryptedMessage + 16, // 56: management.ManagementService.GetServerKey:output_type -> management.ServerKeyResponse + 17, // 57: management.ManagementService.isHealthy:output_type -> management.Empty + 5, // 58: management.ManagementService.GetDeviceAuthorizationFlow:output_type -> management.EncryptedMessage + 5, // 59: management.ManagementService.GetPKCEAuthorizationFlow:output_type -> management.EncryptedMessage + 17, // 60: management.ManagementService.SyncMeta:output_type -> management.Empty + 54, // [54:61] is the sub-list for method output_type + 47, // [47:54] is the sub-list for method input_type + 47, // [47:47] is the sub-list for extension type_name + 47, // [47:47] is the sub-list for extension extendee + 0, // [0:47] is the sub-list for field type_name } func init() { file_management_proto_init() } diff --git a/management/proto/management.proto b/management/proto/management.proto index 2318fc6755a..01330dd485e 100644 --- a/management/proto/management.proto +++ b/management/proto/management.proto @@ -430,6 +430,7 @@ message FirewallRule { RuleAction Action = 3; RuleProtocol Protocol = 4; string Port = 5; + PortInfo portInfo = 6; } message NetworkAddress { From b951fb4aec1e193519a6ae4a045aeaf046c61ae8 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 23 Jan 2025 18:14:16 +0100 Subject: [PATCH 56/59] Use uppercase field name --- management/proto/management.pb.go | 8 ++++---- management/proto/management.proto | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/management/proto/management.pb.go b/management/proto/management.pb.go index 2fc87a0a29e..ae65596758c 100644 --- a/management/proto/management.pb.go +++ b/management/proto/management.pb.go @@ -2624,7 +2624,7 @@ type FirewallRule struct { Action RuleAction `protobuf:"varint,3,opt,name=Action,proto3,enum=management.RuleAction" json:"Action,omitempty"` Protocol RuleProtocol `protobuf:"varint,4,opt,name=Protocol,proto3,enum=management.RuleProtocol" json:"Protocol,omitempty"` Port string `protobuf:"bytes,5,opt,name=Port,proto3" json:"Port,omitempty"` - PortInfo *PortInfo `protobuf:"bytes,6,opt,name=portInfo,proto3" json:"portInfo,omitempty"` + PortInfo *PortInfo `protobuf:"bytes,6,opt,name=PortInfo,proto3" json:"PortInfo,omitempty"` } func (x *FirewallRule) Reset() { @@ -3419,9 +3419,9 @@ var file_management_proto_rawDesc = []byte{ 0x74, 0x2e, 0x52, 0x75, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x08, - 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, + 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x50, 0x6f, 0x72, 0x74, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x38, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x38, 0x0a, 0x0e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x65, 0x74, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x65, 0x74, 0x49, 0x50, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x61, 0x63, 0x18, 0x02, 0x20, @@ -3607,7 +3607,7 @@ var file_management_proto_depIdxs = []int32{ 1, // 39: management.FirewallRule.Direction:type_name -> management.RuleDirection 2, // 40: management.FirewallRule.Action:type_name -> management.RuleAction 0, // 41: management.FirewallRule.Protocol:type_name -> management.RuleProtocol - 40, // 42: management.FirewallRule.portInfo:type_name -> management.PortInfo + 40, // 42: management.FirewallRule.PortInfo:type_name -> management.PortInfo 42, // 43: management.PortInfo.range:type_name -> management.PortInfo.Range 2, // 44: management.RouteFirewallRule.action:type_name -> management.RuleAction 0, // 45: management.RouteFirewallRule.protocol:type_name -> management.RuleProtocol diff --git a/management/proto/management.proto b/management/proto/management.proto index 01330dd485e..9db66ec4d72 100644 --- a/management/proto/management.proto +++ b/management/proto/management.proto @@ -430,7 +430,7 @@ message FirewallRule { RuleAction Action = 3; RuleProtocol Protocol = 4; string Port = 5; - PortInfo portInfo = 6; + PortInfo PortInfo = 6; } message NetworkAddress { From e3d4f9819fd71caad1ec449bfa701dc8621c69fe Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 23 Jan 2025 19:28:42 +0100 Subject: [PATCH 57/59] Add test cases for port ranges in peer ACLs --- .../uspfilter/uspfilter_filter_test.go | 73 ++++++++++++++----- 1 file changed, 54 insertions(+), 19 deletions(-) diff --git a/client/firewall/uspfilter/uspfilter_filter_test.go b/client/firewall/uspfilter/uspfilter_filter_test.go index 56cbe603c7c..d7aebb1aab0 100644 --- a/client/firewall/uspfilter/uspfilter_filter_test.go +++ b/client/firewall/uspfilter/uspfilter_filter_test.go @@ -134,6 +134,60 @@ func TestPeerACLFiltering(t *testing.T) { ruleAction: fw.ActionAccept, shouldBeBlocked: false, }, + { + name: "Allow TCP traffic within port range", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 8080, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolTCP, + ruleDstPort: &fw.Port{IsRange: true, Values: []uint16{8000, 8100}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Block TCP traffic outside port range", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 12345, + dstPort: 7999, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolTCP, + ruleDstPort: &fw.Port{IsRange: true, Values: []uint16{8000, 8100}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: true, + }, + { + name: "Allow TCP traffic with source port range", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 32100, + dstPort: 443, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolTCP, + ruleSrcPort: &fw.Port{IsRange: true, Values: []uint16{32000, 33000}}, + ruleDstPort: &fw.Port{Values: []uint16{443}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: false, + }, + { + name: "Block TCP traffic outside source port range", + srcIP: "100.10.0.1", + dstIP: "100.10.0.100", + proto: fw.ProtocolTCP, + srcPort: 31999, + dstPort: 443, + ruleIP: "100.10.0.1", + ruleProto: fw.ProtocolTCP, + ruleSrcPort: &fw.Port{IsRange: true, Values: []uint16{32000, 33000}}, + ruleDstPort: &fw.Port{Values: []uint16{443}}, + ruleAction: fw.ActionAccept, + shouldBeBlocked: true, + }, } t.Run("Implicit DROP (no rules)", func(t *testing.T) { @@ -636,25 +690,6 @@ func TestRouteACLFiltering(t *testing.T) { }, shouldPass: true, }, - { - name: "Invalid Port Range Configuration", - srcIP: "100.10.0.1", - dstIP: "192.168.1.100", - proto: fw.ProtocolTCP, - srcPort: 12345, - dstPort: 8080, - rule: rule{ - sources: []netip.Prefix{netip.MustParsePrefix("100.10.0.0/16")}, - dest: netip.MustParsePrefix("192.168.1.0/24"), - proto: fw.ProtocolTCP, - dstPort: &fw.Port{ - IsRange: true, - Values: []uint16{8000}, // Invalid: only one value for range - }, - action: fw.ActionAccept, - }, - shouldPass: false, - }, { name: "Edge Case - Port at Range Boundary", srcIP: "100.10.0.1", From 0837864cfcf19eee06649bac97fd8b828acbc585 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Thu, 23 Jan 2025 19:29:25 +0100 Subject: [PATCH 58/59] Fix port --- client/firewall/uspfilter/uspfilter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/firewall/uspfilter/uspfilter.go b/client/firewall/uspfilter/uspfilter.go index 585482059da..757249b2dd5 100644 --- a/client/firewall/uspfilter/uspfilter.go +++ b/client/firewall/uspfilter/uspfilter.go @@ -523,7 +523,7 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]Rule, d *decode return rule.udpHook(packetData), true } - if portsMatch(rule.sPort, uint16(d.tcp.SrcPort)) && portsMatch(rule.dPort, uint16(d.tcp.DstPort)) { + if portsMatch(rule.sPort, uint16(d.udp.SrcPort)) && portsMatch(rule.dPort, uint16(d.udp.DstPort)) { return rule.drop, true } case layers.LayerTypeICMPv4, layers.LayerTypeICMPv6: From 48f58d776cb74cda7031539e5b40d628f6dfbf68 Mon Sep 17 00:00:00 2001 From: Viktor Liu Date: Wed, 29 Jan 2025 22:43:14 +0100 Subject: [PATCH 59/59] Treat the whole localhost range as local IPs --- client/firewall/uspfilter/localip.go | 6 ++++ client/firewall/uspfilter/localip_test.go | 36 +++++++++++++++++++++++ 2 files changed, 42 insertions(+) diff --git a/client/firewall/uspfilter/localip.go b/client/firewall/uspfilter/localip.go index dc0c529be2e..7664b65d554 100644 --- a/client/firewall/uspfilter/localip.go +++ b/client/firewall/uspfilter/localip.go @@ -93,6 +93,12 @@ func (m *localIPManager) UpdateLocalIPs(iface common.IFaceMapper) (err error) { ipv4Set := make(map[string]struct{}) var ipv4Addresses []string + // 127.0.0.0/8 + high := uint16(127) << 8 + for i := uint16(0); i < 256; i++ { + newIPv4Bitmap[high|i] = 0xffffffff + } + if iface != nil { if err := m.processIP(iface.Address().IP, &newIPv4Bitmap, ipv4Set, &ipv4Addresses); err != nil { return err diff --git a/client/firewall/uspfilter/localip_test.go b/client/firewall/uspfilter/localip_test.go index 72a10c970cf..02f41bf4f61 100644 --- a/client/firewall/uspfilter/localip_test.go +++ b/client/firewall/uspfilter/localip_test.go @@ -16,6 +16,42 @@ func TestLocalIPManager(t *testing.T) { testIP net.IP expected bool }{ + { + name: "Localhost range", + setupAddr: iface.WGAddress{ + IP: net.ParseIP("192.168.1.1"), + Network: &net.IPNet{ + IP: net.ParseIP("192.168.1.0"), + Mask: net.CIDRMask(24, 32), + }, + }, + testIP: net.ParseIP("127.0.0.2"), + expected: true, + }, + { + name: "Localhost standard address", + setupAddr: iface.WGAddress{ + IP: net.ParseIP("192.168.1.1"), + Network: &net.IPNet{ + IP: net.ParseIP("192.168.1.0"), + Mask: net.CIDRMask(24, 32), + }, + }, + testIP: net.ParseIP("127.0.0.1"), + expected: true, + }, + { + name: "Localhost range edge", + setupAddr: iface.WGAddress{ + IP: net.ParseIP("192.168.1.1"), + Network: &net.IPNet{ + IP: net.ParseIP("192.168.1.0"), + Mask: net.CIDRMask(24, 32), + }, + }, + testIP: net.ParseIP("127.255.255.255"), + expected: true, + }, { name: "Local IP matches", setupAddr: iface.WGAddress{