diff options
-rw-r--r-- | pkg/sentry/fs/proc/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/fs/proc/net.go | 201 | ||||
-rw-r--r-- | pkg/sentry/socket/epsocket/epsocket.go | 25 | ||||
-rw-r--r-- | pkg/tcpip/transport/udp/endpoint.go | 61 | ||||
-rw-r--r-- | pkg/tcpip/transport/udp/endpoint_state.go | 4 | ||||
-rw-r--r-- | pkg/tcpip/transport/udp/forwarder.go | 2 | ||||
-rw-r--r-- | test/syscalls/BUILD | 4 | ||||
-rw-r--r-- | test/syscalls/linux/BUILD | 15 | ||||
-rw-r--r-- | test/syscalls/linux/ip_socket_test_util.cc | 10 | ||||
-rw-r--r-- | test/syscalls/linux/ip_socket_test_util.h | 25 | ||||
-rw-r--r-- | test/syscalls/linux/proc_net_tcp.cc | 65 | ||||
-rw-r--r-- | test/syscalls/linux/proc_net_udp.cc | 309 | ||||
-rw-r--r-- | test/util/fs_util.cc | 9 | ||||
-rw-r--r-- | test/util/fs_util.h | 3 |
14 files changed, 633 insertions, 101 deletions
diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index 70ed854a8..c7599d1f6 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -31,7 +31,6 @@ go_library( visibility = ["//pkg/sentry:internal"], deps = [ "//pkg/abi/linux", - "//pkg/binary", "//pkg/log", "//pkg/sentry/context", "//pkg/sentry/fs", diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index 9adb23608..5e28982c5 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -17,10 +17,10 @@ package proc import ( "bytes" "fmt" + "io" "time" "gvisor.dev/gvisor/pkg/abi/linux" - "gvisor.dev/gvisor/pkg/binary" "gvisor.dev/gvisor/pkg/log" "gvisor.dev/gvisor/pkg/sentry/context" "gvisor.dev/gvisor/pkg/sentry/fs" @@ -28,9 +28,11 @@ import ( "gvisor.dev/gvisor/pkg/sentry/fs/ramfs" "gvisor.dev/gvisor/pkg/sentry/inet" "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/socket" "gvisor.dev/gvisor/pkg/sentry/socket/unix" "gvisor.dev/gvisor/pkg/sentry/socket/unix/transport" + "gvisor.dev/gvisor/pkg/sentry/usermem" ) // newNet creates a new proc net entry. @@ -57,9 +59,8 @@ func (p *proc) newNetDir(ctx context.Context, k *kernel.Kernel, msrc *fs.MountSo "ptype": newStaticProcInode(ctx, msrc, []byte("Type Device Function")), "route": newStaticProcInode(ctx, msrc, []byte("Iface Destination Gateway Flags RefCnt Use Metric Mask MTU Window IRTT")), "tcp": seqfile.NewSeqFileInode(ctx, &netTCP{k: k}, msrc), - "udp": newStaticProcInode(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops")), - - "unix": seqfile.NewSeqFileInode(ctx, &netUnix{k: k}, msrc), + "udp": seqfile.NewSeqFileInode(ctx, &netUDP{k: k}, msrc), + "unix": seqfile.NewSeqFileInode(ctx, &netUnix{k: k}, msrc), } if s.SupportsIPv6() { @@ -216,7 +217,7 @@ func (n *netUnix) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]s for _, se := range n.k.ListSockets() { s := se.Sock.Get() if s == nil { - log.Debugf("Couldn't resolve weakref %v in socket table, racing with destruction?", se.Sock) + log.Debugf("Couldn't resolve weakref with ID %v in socket table, racing with destruction?", se.ID) continue } sfile := s.(*fs.File) @@ -297,6 +298,42 @@ func (n *netUnix) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]s return data, 0 } +func networkToHost16(n uint16) uint16 { + // n is in network byte order, so is big-endian. The most-significant byte + // should be stored in the lower address. + // + // We manually inline binary.BigEndian.Uint16() because Go does not support + // non-primitive consts, so binary.BigEndian is a (mutable) var, so calls to + // binary.BigEndian.Uint16() require a read of binary.BigEndian and an + // interface method call, defeating inlining. + buf := [2]byte{byte(n >> 8 & 0xff), byte(n & 0xff)} + return usermem.ByteOrder.Uint16(buf[:]) +} + +func writeInetAddr(w io.Writer, a linux.SockAddrInet) { + // linux.SockAddrInet.Port is stored in the network byte order and is + // printed like a number in host byte order. Note that all numbers in host + // byte order are printed with the most-significant byte first when + // formatted with %X. See get_tcp4_sock() and udp4_format_sock() in Linux. + port := networkToHost16(a.Port) + + // linux.SockAddrInet.Addr is stored as a byte slice in big-endian order + // (i.e. most-significant byte in index 0). Linux represents this as a + // __be32 which is a typedef for an unsigned int, and is printed with + // %X. This means that for a little-endian machine, Linux prints the + // least-significant byte of the address first. To emulate this, we first + // invert the byte order for the address using usermem.ByteOrder.Uint32, + // which makes it have the equivalent encoding to a __be32 on a little + // endian machine. Note that this operation is a no-op on a big endian + // machine. Then similar to Linux, we format it with %X, which will print + // the most-significant byte of the __be32 address first, which is now + // actually the least-significant byte of the original address in + // linux.SockAddrInet.Addr on little endian machines, due to the conversion. + addr := usermem.ByteOrder.Uint32(a.Addr[:]) + + fmt.Fprintf(w, "%08X:%04X ", addr, port) +} + // netTCP implements seqfile.SeqSource for /proc/net/tcp. // // +stateify savable @@ -311,6 +348,9 @@ func (*netTCP) NeedsUpdate(generation int64) bool { // ReadSeqFileData implements seqfile.SeqSource.ReadSeqFileData. func (n *netTCP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]seqfile.SeqData, int64) { + // t may be nil here if our caller is not part of a task goroutine. This can + // happen for example if we're here for "sentryctl cat". When t is nil, + // degrade gracefully and retrieve what we can. t := kernel.TaskFromContext(ctx) if h != nil { @@ -321,7 +361,7 @@ func (n *netTCP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]se for _, se := range n.k.ListSockets() { s := se.Sock.Get() if s == nil { - log.Debugf("Couldn't resolve weakref %+v in socket table, racing with destruction?", se.Sock) + log.Debugf("Couldn't resolve weakref with ID %v in socket table, racing with destruction?", se.ID) continue } sfile := s.(*fs.File) @@ -343,27 +383,23 @@ func (n *netTCP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]se // Field: sl; entry number. fmt.Fprintf(&buf, "%4d: ", se.ID) - portBuf := make([]byte, 2) - // Field: local_adddress. var localAddr linux.SockAddrInet - if local, _, err := sops.GetSockName(t); err == nil { - localAddr = *local.(*linux.SockAddrInet) + if t != nil { + if local, _, err := sops.GetSockName(t); err == nil { + localAddr = *local.(*linux.SockAddrInet) + } } - binary.LittleEndian.PutUint16(portBuf, localAddr.Port) - fmt.Fprintf(&buf, "%08X:%04X ", - binary.LittleEndian.Uint32(localAddr.Addr[:]), - portBuf) + writeInetAddr(&buf, localAddr) // Field: rem_address. var remoteAddr linux.SockAddrInet - if remote, _, err := sops.GetPeerName(t); err == nil { - remoteAddr = *remote.(*linux.SockAddrInet) + if t != nil { + if remote, _, err := sops.GetPeerName(t); err == nil { + remoteAddr = *remote.(*linux.SockAddrInet) + } } - binary.LittleEndian.PutUint16(portBuf, remoteAddr.Port) - fmt.Fprintf(&buf, "%08X:%04X ", - binary.LittleEndian.Uint32(remoteAddr.Addr[:]), - portBuf) + writeInetAddr(&buf, remoteAddr) // Field: state; socket state. fmt.Fprintf(&buf, "%02X ", sops.State()) @@ -386,7 +422,8 @@ func (n *netTCP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]se log.Warningf("Failed to retrieve unstable attr for socket file: %v", err) fmt.Fprintf(&buf, "%5d ", 0) } else { - fmt.Fprintf(&buf, "%5d ", uint32(uattr.Owner.UID.In(t.UserNamespace()).OrOverflow())) + creds := auth.CredentialsFromContext(ctx) + fmt.Fprintf(&buf, "%5d ", uint32(uattr.Owner.UID.In(creds.UserNamespace).OrOverflow())) } // Field: timeout; number of unanswered 0-window probes. @@ -438,3 +475,125 @@ func (n *netTCP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]se } return data, 0 } + +// netUDP implements seqfile.SeqSource for /proc/net/udp. +// +// +stateify savable +type netUDP struct { + k *kernel.Kernel +} + +// NeedsUpdate implements seqfile.SeqSource.NeedsUpdate. +func (*netUDP) NeedsUpdate(generation int64) bool { + return true +} + +// ReadSeqFileData implements seqfile.SeqSource.ReadSeqFileData. +func (n *netUDP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]seqfile.SeqData, int64) { + // t may be nil here if our caller is not part of a task goroutine. This can + // happen for example if we're here for "sentryctl cat". When t is nil, + // degrade gracefully and retrieve what we can. + t := kernel.TaskFromContext(ctx) + + if h != nil { + return nil, 0 + } + + var buf bytes.Buffer + for _, se := range n.k.ListSockets() { + s := se.Sock.Get() + if s == nil { + log.Debugf("Couldn't resolve weakref with ID %v in socket table, racing with destruction?", se.ID) + continue + } + sfile := s.(*fs.File) + sops, ok := sfile.FileOperations.(socket.Socket) + if !ok { + panic(fmt.Sprintf("Found non-socket file in socket table: %+v", sfile)) + } + if family, stype, _ := sops.Type(); family != linux.AF_INET || stype != linux.SOCK_DGRAM { + s.DecRef() + // Not udp4 socket. + continue + } + + // For Linux's implementation, see net/ipv4/udp.c:udp4_format_sock(). + + // Field: sl; entry number. + fmt.Fprintf(&buf, "%5d: ", se.ID) + + // Field: local_adddress. + var localAddr linux.SockAddrInet + if t != nil { + if local, _, err := sops.GetSockName(t); err == nil { + localAddr = *local.(*linux.SockAddrInet) + } + } + writeInetAddr(&buf, localAddr) + + // Field: rem_address. + var remoteAddr linux.SockAddrInet + if t != nil { + if remote, _, err := sops.GetPeerName(t); err == nil { + remoteAddr = *remote.(*linux.SockAddrInet) + } + } + writeInetAddr(&buf, remoteAddr) + + // Field: state; socket state. + fmt.Fprintf(&buf, "%02X ", sops.State()) + + // Field: tx_queue, rx_queue; number of packets in the transmit and + // receive queue. Unimplemented. + fmt.Fprintf(&buf, "%08X:%08X ", 0, 0) + + // Field: tr, tm->when. Always 0 for UDP. + fmt.Fprintf(&buf, "%02X:%08X ", 0, 0) + + // Field: retrnsmt. Always 0 for UDP. + fmt.Fprintf(&buf, "%08X ", 0) + + // Field: uid. + uattr, err := sfile.Dirent.Inode.UnstableAttr(ctx) + if err != nil { + log.Warningf("Failed to retrieve unstable attr for socket file: %v", err) + fmt.Fprintf(&buf, "%5d ", 0) + } else { + creds := auth.CredentialsFromContext(ctx) + fmt.Fprintf(&buf, "%5d ", uint32(uattr.Owner.UID.In(creds.UserNamespace).OrOverflow())) + } + + // Field: timeout. Always 0 for UDP. + fmt.Fprintf(&buf, "%8d ", 0) + + // Field: inode. + fmt.Fprintf(&buf, "%8d ", sfile.InodeID()) + + // Field: ref; reference count on the socket inode. Don't count the ref + // we obtain while deferencing the weakref to this socket. + fmt.Fprintf(&buf, "%d ", sfile.ReadRefs()-1) + + // Field: Socket struct address. Redacted due to the same reason as + // the 'Num' field in /proc/net/unix, see netUnix.ReadSeqFileData. + fmt.Fprintf(&buf, "%#016p ", (*socket.Socket)(nil)) + + // Field: drops; number of dropped packets. Unimplemented. + fmt.Fprintf(&buf, "%d", 0) + + fmt.Fprintf(&buf, "\n") + + s.DecRef() + } + + data := []seqfile.SeqData{ + { + Buf: []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops \n"), + Handle: n, + }, + { + Buf: buf.Bytes(), + Handle: n, + }, + } + return data, 0 +} diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 635042263..def29646e 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -27,12 +27,14 @@ package epsocket import ( "bytes" "math" + "reflect" "sync" "syscall" "time" "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/binary" + "gvisor.dev/gvisor/pkg/log" "gvisor.dev/gvisor/pkg/metric" "gvisor.dev/gvisor/pkg/sentry/arch" "gvisor.dev/gvisor/pkg/sentry/context" @@ -52,6 +54,7 @@ import ( "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" + "gvisor.dev/gvisor/pkg/tcpip/transport/udp" "gvisor.dev/gvisor/pkg/waiter" ) @@ -2421,7 +2424,8 @@ func (s *SocketOperations) State() uint32 { return 0 } - if !s.isPacketBased() { + switch { + case s.skType == linux.SOCK_STREAM && s.protocol == 0 || s.protocol == syscall.IPPROTO_TCP: // TCP socket. switch tcp.EndpointState(s.Endpoint.State()) { case tcp.StateEstablished: @@ -2450,9 +2454,26 @@ func (s *SocketOperations) State() uint32 { // Internal or unknown state. return 0 } + case s.skType == linux.SOCK_DGRAM && s.protocol == 0 || s.protocol == syscall.IPPROTO_UDP: + // UDP socket. + switch udp.EndpointState(s.Endpoint.State()) { + case udp.StateInitial, udp.StateBound, udp.StateClosed: + return linux.TCP_CLOSE + case udp.StateConnected: + return linux.TCP_ESTABLISHED + default: + return 0 + } + case s.skType == linux.SOCK_DGRAM && s.protocol == syscall.IPPROTO_ICMP || s.protocol == syscall.IPPROTO_ICMPV6: + // TODO(b/112063468): Export states for ICMP sockets. + case s.skType == linux.SOCK_RAW: + // TODO(b/112063468): Export states for raw sockets. + default: + // Unknown transport protocol, how did we make this socket? + log.Warningf("Unknown transport protocol for an existing socket: family=%v, type=%v, protocol=%v, internal type %v", s.family, s.skType, s.protocol, reflect.TypeOf(s.Endpoint).Elem()) + return 0 } - // TODO(b/112063468): Export states for UDP, ICMP, and raw sockets. return 0 } diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index ac5905772..66455ef46 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -37,13 +37,17 @@ type udpPacket struct { views [8]buffer.View `state:"nosave"` } -type endpointState int +// EndpointState represents the state of a UDP endpoint. +type EndpointState uint32 +// Endpoint states. Note that are represented in a netstack-specific manner and +// may not be meaningful externally. Specifically, they need to be translated to +// Linux's representation for these states if presented to userspace. const ( - stateInitial endpointState = iota - stateBound - stateConnected - stateClosed + StateInitial EndpointState = iota + StateBound + StateConnected + StateClosed ) // endpoint represents a UDP endpoint. This struct serves as the interface @@ -74,7 +78,7 @@ type endpoint struct { mu sync.RWMutex `state:"nosave"` sndBufSize int id stack.TransportEndpointID - state endpointState + state EndpointState bindNICID tcpip.NICID regNICID tcpip.NICID route stack.Route `state:"manual"` @@ -140,7 +144,7 @@ func (e *endpoint) Close() { e.shutdownFlags = tcpip.ShutdownRead | tcpip.ShutdownWrite switch e.state { - case stateBound, stateConnected: + case StateBound, StateConnected: e.stack.UnregisterTransportEndpoint(e.regNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e) e.stack.ReleasePort(e.effectiveNetProtos, ProtocolNumber, e.id.LocalAddress, e.id.LocalPort) } @@ -163,7 +167,7 @@ func (e *endpoint) Close() { e.route.Release() // Update the state. - e.state = stateClosed + e.state = StateClosed e.mu.Unlock() @@ -211,11 +215,11 @@ func (e *endpoint) Read(addr *tcpip.FullAddress) (buffer.View, tcpip.ControlMess // Returns true for retry if preparation should be retried. func (e *endpoint) prepareForWrite(to *tcpip.FullAddress) (retry bool, err *tcpip.Error) { switch e.state { - case stateInitial: - case stateConnected: + case StateInitial: + case StateConnected: return false, nil - case stateBound: + case StateBound: if to == nil { return false, tcpip.ErrDestinationRequired } @@ -232,7 +236,7 @@ func (e *endpoint) prepareForWrite(to *tcpip.FullAddress) (retry bool, err *tcpi // The state changed when we released the shared locked and re-acquired // it in exclusive mode. Try again. - if e.state != stateInitial { + if e.state != StateInitial { return true, nil } @@ -322,7 +326,7 @@ func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (int64, <-cha defer e.mu.Unlock() // Recheck state after lock was re-acquired. - if e.state != stateConnected { + if e.state != StateConnected { return 0, nil, tcpip.ErrInvalidEndpointState } } @@ -400,7 +404,7 @@ func (e *endpoint) SetSockOpt(opt interface{}) *tcpip.Error { defer e.mu.Unlock() // We only allow this to be set when we're in the initial state. - if e.state != stateInitial { + if e.state != StateInitial { return tcpip.ErrInvalidEndpointState } @@ -726,7 +730,7 @@ func (e *endpoint) Disconnect() *tcpip.Error { e.mu.Lock() defer e.mu.Unlock() - if e.state != stateConnected { + if e.state != StateConnected { return nil } id := stack.TransportEndpointID{} @@ -741,9 +745,9 @@ func (e *endpoint) Disconnect() *tcpip.Error { if err != nil { return err } - e.state = stateBound + e.state = StateBound } else { - e.state = stateInitial + e.state = StateInitial } e.stack.UnregisterTransportEndpoint(e.regNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e) @@ -772,8 +776,8 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { nicid := addr.NIC var localPort uint16 switch e.state { - case stateInitial: - case stateBound, stateConnected: + case StateInitial: + case StateBound, StateConnected: localPort = e.id.LocalPort if e.bindNICID == 0 { break @@ -801,7 +805,7 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { RemoteAddress: r.RemoteAddress, } - if e.state == stateInitial { + if e.state == StateInitial { id.LocalAddress = r.LocalAddress } @@ -832,7 +836,7 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { e.regNICID = nicid e.effectiveNetProtos = netProtos - e.state = stateConnected + e.state = StateConnected e.rcvMu.Lock() e.rcvReady = true @@ -854,7 +858,7 @@ func (e *endpoint) Shutdown(flags tcpip.ShutdownFlags) *tcpip.Error { // A socket in the bound state can still receive multicast messages, // so we need to notify waiters on shutdown. - if e.state != stateBound && e.state != stateConnected { + if e.state != StateBound && e.state != StateConnected { return tcpip.ErrNotConnected } @@ -903,7 +907,7 @@ func (e *endpoint) registerWithStack(nicid tcpip.NICID, netProtos []tcpip.Networ func (e *endpoint) bindLocked(addr tcpip.FullAddress) *tcpip.Error { // Don't allow binding once endpoint is not in the initial state // anymore. - if e.state != stateInitial { + if e.state != StateInitial { return tcpip.ErrInvalidEndpointState } @@ -946,7 +950,7 @@ func (e *endpoint) bindLocked(addr tcpip.FullAddress) *tcpip.Error { e.effectiveNetProtos = netProtos // Mark endpoint as bound. - e.state = stateBound + e.state = StateBound e.rcvMu.Lock() e.rcvReady = true @@ -989,7 +993,7 @@ func (e *endpoint) GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) { e.mu.RLock() defer e.mu.RUnlock() - if e.state != stateConnected { + if e.state != StateConnected { return tcpip.FullAddress{}, tcpip.ErrNotConnected } @@ -1069,10 +1073,11 @@ func (e *endpoint) HandlePacket(r *stack.Route, id stack.TransportEndpointID, vv func (e *endpoint) HandleControlPacket(id stack.TransportEndpointID, typ stack.ControlType, extra uint32, vv buffer.VectorisedView) { } -// State implements socket.Socket.State. +// State implements tcpip.Endpoint.State. func (e *endpoint) State() uint32 { - // TODO(b/112063468): Translate internal state to values returned by Linux. - return 0 + e.mu.Lock() + defer e.mu.Unlock() + return uint32(e.state) } func isBroadcastOrMulticast(a tcpip.Address) bool { diff --git a/pkg/tcpip/transport/udp/endpoint_state.go b/pkg/tcpip/transport/udp/endpoint_state.go index 5cbb56120..be46e6d4e 100644 --- a/pkg/tcpip/transport/udp/endpoint_state.go +++ b/pkg/tcpip/transport/udp/endpoint_state.go @@ -77,7 +77,7 @@ func (e *endpoint) Resume(s *stack.Stack) { } } - if e.state != stateBound && e.state != stateConnected { + if e.state != StateBound && e.state != StateConnected { return } @@ -92,7 +92,7 @@ func (e *endpoint) Resume(s *stack.Stack) { } var err *tcpip.Error - if e.state == stateConnected { + if e.state == StateConnected { e.route, err = e.stack.FindRoute(e.regNICID, e.id.LocalAddress, e.id.RemoteAddress, netProto, e.multicastLoop) if err != nil { panic(err) diff --git a/pkg/tcpip/transport/udp/forwarder.go b/pkg/tcpip/transport/udp/forwarder.go index a874fc9fd..a9edc2c8d 100644 --- a/pkg/tcpip/transport/udp/forwarder.go +++ b/pkg/tcpip/transport/udp/forwarder.go @@ -84,7 +84,7 @@ func (r *ForwarderRequest) CreateEndpoint(queue *waiter.Queue) (tcpip.Endpoint, ep.dstPort = r.id.RemotePort ep.regNICID = r.route.NICID() - ep.state = stateConnected + ep.state = StateConnected ep.rcvMu.Lock() ep.rcvReady = true diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index ccae4925f..6947ddc25 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -691,6 +691,10 @@ syscall_test( syscall_test(test = "//test/syscalls/linux:proc_net_unix_test") +syscall_test(test = "//test/syscalls/linux:proc_net_tcp_test") + +syscall_test(test = "//test/syscalls/linux:proc_net_udp_test") + go_binary( name = "syscall_test_runner", srcs = ["syscall_test_runner.go"], diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index 88f3bfcb3..1ce38c929 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -3472,3 +3472,18 @@ cc_binary( "@com_google_googletest//:gtest", ], ) + +cc_binary( + name = "proc_net_udp_test", + testonly = 1, + srcs = ["proc_net_udp.cc"], + linkstatic = 1, + deps = [ + ":ip_socket_test_util", + "//test/util:file_descriptor", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/strings", + "@com_google_googletest//:gtest", + ], +) diff --git a/test/syscalls/linux/ip_socket_test_util.cc b/test/syscalls/linux/ip_socket_test_util.cc index c73262e72..410b42a47 100644 --- a/test/syscalls/linux/ip_socket_test_util.cc +++ b/test/syscalls/linux/ip_socket_test_util.cc @@ -23,6 +23,16 @@ namespace gvisor { namespace testing { +uint32_t IPFromInetSockaddr(const struct sockaddr* addr) { + auto* in_addr = reinterpret_cast<const struct sockaddr_in*>(addr); + return in_addr->sin_addr.s_addr; +} + +uint16_t PortFromInetSockaddr(const struct sockaddr* addr) { + auto* in_addr = reinterpret_cast<const struct sockaddr_in*>(addr); + return ntohs(in_addr->sin_port); +} + PosixErrorOr<int> InterfaceIndex(std::string name) { // TODO(igudger): Consider using netlink. ifreq req = {}; diff --git a/test/syscalls/linux/ip_socket_test_util.h b/test/syscalls/linux/ip_socket_test_util.h index b498a053d..3d36b9620 100644 --- a/test/syscalls/linux/ip_socket_test_util.h +++ b/test/syscalls/linux/ip_socket_test_util.h @@ -26,6 +26,31 @@ namespace gvisor { namespace testing { +// Possible values of the "st" field in a /proc/net/{tcp,udp} entry. Source: +// Linux kernel, include/net/tcp_states.h. +enum { + TCP_ESTABLISHED = 1, + TCP_SYN_SENT, + TCP_SYN_RECV, + TCP_FIN_WAIT1, + TCP_FIN_WAIT2, + TCP_TIME_WAIT, + TCP_CLOSE, + TCP_CLOSE_WAIT, + TCP_LAST_ACK, + TCP_LISTEN, + TCP_CLOSING, + TCP_NEW_SYN_RECV, + + TCP_MAX_STATES +}; + +// Extracts the IP address from an inet sockaddr in network byte order. +uint32_t IPFromInetSockaddr(const struct sockaddr* addr); + +// Extracts the port from an inet sockaddr in host byte order. +uint16_t PortFromInetSockaddr(const struct sockaddr* addr); + // InterfaceIndex returns the index of the named interface. PosixErrorOr<int> InterfaceIndex(std::string name); diff --git a/test/syscalls/linux/proc_net_tcp.cc b/test/syscalls/linux/proc_net_tcp.cc index 498f62d9c..f6d7ad0bb 100644 --- a/test/syscalls/linux/proc_net_tcp.cc +++ b/test/syscalls/linux/proc_net_tcp.cc @@ -38,25 +38,6 @@ constexpr char kProcNetTCPHeader[] = "retrnsmt uid timeout inode " " "; -// Possible values of the "st" field in a /proc/net/tcp entry. Source: Linux -// kernel, include/net/tcp_states.h. -enum { - TCP_ESTABLISHED = 1, - TCP_SYN_SENT, - TCP_SYN_RECV, - TCP_FIN_WAIT1, - TCP_FIN_WAIT2, - TCP_TIME_WAIT, - TCP_CLOSE, - TCP_CLOSE_WAIT, - TCP_LAST_ACK, - TCP_LISTEN, - TCP_CLOSING, - TCP_NEW_SYN_RECV, - - TCP_MAX_STATES -}; - // TCPEntry represents a single entry from /proc/net/tcp. struct TCPEntry { uint32_t local_addr; @@ -70,42 +51,35 @@ struct TCPEntry { uint64_t inode; }; -uint32_t IP(const struct sockaddr* addr) { - auto* in_addr = reinterpret_cast<const struct sockaddr_in*>(addr); - return in_addr->sin_addr.s_addr; -} - -uint16_t Port(const struct sockaddr* addr) { - auto* in_addr = reinterpret_cast<const struct sockaddr_in*>(addr); - return ntohs(in_addr->sin_port); -} - // Finds the first entry in 'entries' for which 'predicate' returns true. -// Returns true on match, and sets 'match' to point to the matching entry. -bool FindBy(std::vector<TCPEntry> entries, TCPEntry* match, +// Returns true on match, and sets 'match' to a copy of the matching entry. If +// 'match' is null, it's ignored. +bool FindBy(const std::vector<TCPEntry>& entries, TCPEntry* match, std::function<bool(const TCPEntry&)> predicate) { - for (int i = 0; i < entries.size(); ++i) { - if (predicate(entries[i])) { - *match = entries[i]; + for (const TCPEntry& entry : entries) { + if (predicate(entry)) { + if (match != nullptr) { + *match = entry; + } return true; } } return false; } -bool FindByLocalAddr(std::vector<TCPEntry> entries, TCPEntry* match, +bool FindByLocalAddr(const std::vector<TCPEntry>& entries, TCPEntry* match, const struct sockaddr* addr) { - uint32_t host = IP(addr); - uint16_t port = Port(addr); + uint32_t host = IPFromInetSockaddr(addr); + uint16_t port = PortFromInetSockaddr(addr); return FindBy(entries, match, [host, port](const TCPEntry& e) { return (e.local_addr == host && e.local_port == port); }); } -bool FindByRemoteAddr(std::vector<TCPEntry> entries, TCPEntry* match, +bool FindByRemoteAddr(const std::vector<TCPEntry>& entries, TCPEntry* match, const struct sockaddr* addr) { - uint32_t host = IP(addr); - uint16_t port = Port(addr); + uint32_t host = IPFromInetSockaddr(addr); + uint16_t port = PortFromInetSockaddr(addr); return FindBy(entries, match, [host, port](const TCPEntry& e) { return (e.remote_addr == host && e.remote_port == port); }); @@ -120,7 +94,7 @@ PosixErrorOr<std::vector<TCPEntry>> ProcNetTCPEntries() { std::vector<TCPEntry> entries; std::vector<std::string> lines = StrSplit(content, '\n'); std::cerr << "<contents of /proc/net/tcp>" << std::endl; - for (std::string line : lines) { + for (const std::string& line : lines) { std::cerr << line << std::endl; if (!found_header) { @@ -204,9 +178,8 @@ TEST(ProcNetTCP, BindAcceptConnect) { EXPECT_EQ(entries.size(), 2); } - TCPEntry e; - EXPECT_TRUE(FindByLocalAddr(entries, &e, sockets->first_addr())); - EXPECT_TRUE(FindByRemoteAddr(entries, &e, sockets->first_addr())); + EXPECT_TRUE(FindByLocalAddr(entries, nullptr, sockets->first_addr())); + EXPECT_TRUE(FindByRemoteAddr(entries, nullptr, sockets->first_addr())); } TEST(ProcNetTCP, InodeReasonable) { @@ -261,8 +234,8 @@ TEST(ProcNetTCP, State) { FileDescriptor accepted = ASSERT_NO_ERRNO_AND_VALUE(Accept(server->get(), nullptr, nullptr)); - const uint32_t accepted_local_host = IP(&addr); - const uint16_t accepted_local_port = Port(&addr); + const uint32_t accepted_local_host = IPFromInetSockaddr(&addr); + const uint16_t accepted_local_port = PortFromInetSockaddr(&addr); entries = ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); TCPEntry accepted_entry; diff --git a/test/syscalls/linux/proc_net_udp.cc b/test/syscalls/linux/proc_net_udp.cc new file mode 100644 index 000000000..369df8e0e --- /dev/null +++ b/test/syscalls/linux/proc_net_udp.cc @@ -0,0 +1,309 @@ +// Copyright 2019 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include <sys/socket.h> +#include <sys/stat.h> +#include <sys/types.h> +#include <unistd.h> + +#include "gtest/gtest.h" +#include "gtest/gtest.h" +#include "absl/strings/numbers.h" +#include "absl/strings/str_join.h" +#include "absl/strings/str_split.h" +#include "test/syscalls/linux/ip_socket_test_util.h" +#include "test/util/file_descriptor.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { +namespace { + +using absl::StrCat; +using absl::StrFormat; +using absl::StrSplit; + +constexpr char kProcNetUDPHeader[] = + " sl local_address rem_address st tx_queue rx_queue tr tm->when " + "retrnsmt uid timeout inode ref pointer drops "; + +// UDPEntry represents a single entry from /proc/net/udp. +struct UDPEntry { + uint32_t local_addr; + uint16_t local_port; + + uint32_t remote_addr; + uint16_t remote_port; + + uint64_t state; + uint64_t uid; + uint64_t inode; +}; + +std::string DescribeFirstInetSocket(const SocketPair& sockets) { + const struct sockaddr* addr = sockets.first_addr(); + return StrFormat("First test socket: fd:%d %8X:%4X", sockets.first_fd(), + IPFromInetSockaddr(addr), PortFromInetSockaddr(addr)); +} + +std::string DescribeSecondInetSocket(const SocketPair& sockets) { + const struct sockaddr* addr = sockets.second_addr(); + return StrFormat("Second test socket fd:%d %8X:%4X", sockets.second_fd(), + IPFromInetSockaddr(addr), PortFromInetSockaddr(addr)); +} + +// Finds the first entry in 'entries' for which 'predicate' returns true. +// Returns true on match, and set 'match' to a copy of the matching entry. If +// 'match' is null, it's ignored. +bool FindBy(const std::vector<UDPEntry>& entries, UDPEntry* match, + std::function<bool(const UDPEntry&)> predicate) { + for (const UDPEntry& entry : entries) { + if (predicate(entry)) { + if (match != nullptr) { + *match = entry; + } + return true; + } + } + return false; +} + +bool FindByLocalAddr(const std::vector<UDPEntry>& entries, UDPEntry* match, + const struct sockaddr* addr) { + uint32_t host = IPFromInetSockaddr(addr); + uint16_t port = PortFromInetSockaddr(addr); + return FindBy(entries, match, [host, port](const UDPEntry& e) { + return (e.local_addr == host && e.local_port == port); + }); +} + +bool FindByRemoteAddr(const std::vector<UDPEntry>& entries, UDPEntry* match, + const struct sockaddr* addr) { + uint32_t host = IPFromInetSockaddr(addr); + uint16_t port = PortFromInetSockaddr(addr); + return FindBy(entries, match, [host, port](const UDPEntry& e) { + return (e.remote_addr == host && e.remote_port == port); + }); +} + +PosixErrorOr<uint64_t> InodeFromSocketFD(int fd) { + ASSIGN_OR_RETURN_ERRNO(struct stat s, Fstat(fd)); + if (!S_ISSOCK(s.st_mode)) { + return PosixError(EINVAL, StrFormat("FD %d is not a socket", fd)); + } + return s.st_ino; +} + +PosixErrorOr<bool> FindByFD(const std::vector<UDPEntry>& entries, + UDPEntry* match, int fd) { + ASSIGN_OR_RETURN_ERRNO(uint64_t inode, InodeFromSocketFD(fd)); + return FindBy(entries, match, + [inode](const UDPEntry& e) { return (e.inode == inode); }); +} + +// Returns a parsed representation of /proc/net/udp entries. +PosixErrorOr<std::vector<UDPEntry>> ProcNetUDPEntries() { + std::string content; + RETURN_IF_ERRNO(GetContents("/proc/net/udp", &content)); + + bool found_header = false; + std::vector<UDPEntry> entries; + std::vector<std::string> lines = StrSplit(content, '\n'); + std::cerr << "<contents of /proc/net/udp>" << std::endl; + for (const std::string& line : lines) { + std::cerr << line << std::endl; + + if (!found_header) { + EXPECT_EQ(line, kProcNetUDPHeader); + found_header = true; + continue; + } + if (line.empty()) { + continue; + } + + // Parse a single entry from /proc/net/udp. + // + // Example entries: + // + // clang-format off + // + // sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops + // 3503: 0100007F:0035 00000000:0000 07 00000000:00000000 00:00000000 00000000 0 0 33317 2 0000000000000000 0 + // 3518: 00000000:0044 00000000:0000 07 00000000:00000000 00:00000000 00000000 0 0 40394 2 0000000000000000 0 + // ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ + // 0 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 + // + // clang-format on + + UDPEntry entry; + std::vector<std::string> fields = + StrSplit(line, absl::ByAnyChar(": "), absl::SkipEmpty()); + + ASSIGN_OR_RETURN_ERRNO(entry.local_addr, AtoiBase(fields[1], 16)); + ASSIGN_OR_RETURN_ERRNO(entry.local_port, AtoiBase(fields[2], 16)); + + ASSIGN_OR_RETURN_ERRNO(entry.remote_addr, AtoiBase(fields[3], 16)); + ASSIGN_OR_RETURN_ERRNO(entry.remote_port, AtoiBase(fields[4], 16)); + + ASSIGN_OR_RETURN_ERRNO(entry.state, AtoiBase(fields[5], 16)); + ASSIGN_OR_RETURN_ERRNO(entry.uid, Atoi<uint64_t>(fields[11])); + ASSIGN_OR_RETURN_ERRNO(entry.inode, Atoi<uint64_t>(fields[13])); + + // Linux shares internal data structures between TCP and UDP sockets. The + // proc entries for UDP sockets share some fields with TCP sockets, but + // these fields should always be zero as they're not meaningful for UDP + // sockets. + EXPECT_EQ(fields[8], "00") << StrFormat("sl:%s, tr", fields[0]); + EXPECT_EQ(fields[9], "00000000") << StrFormat("sl:%s, tm->when", fields[0]); + EXPECT_EQ(fields[10], "00000000") + << StrFormat("sl:%s, retrnsmt", fields[0]); + EXPECT_EQ(fields[12], "0") << StrFormat("sl:%s, timeout", fields[0]); + + entries.push_back(entry); + } + std::cerr << "<end of /proc/net/udp>" << std::endl; + + return entries; +} + +TEST(ProcNetUDP, Exists) { + const std::string content = + ASSERT_NO_ERRNO_AND_VALUE(GetContents("/proc/net/udp")); + const std::string header_line = StrCat(kProcNetUDPHeader, "\n"); + EXPECT_THAT(content, ::testing::StartsWith(header_line)); +} + +TEST(ProcNetUDP, EntryUID) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4UDPBidirectionalBindSocketPair(0).Create()); + std::vector<UDPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + UDPEntry e; + ASSERT_TRUE(FindByLocalAddr(entries, &e, sockets->first_addr())) + << DescribeFirstInetSocket(*sockets); + EXPECT_EQ(e.uid, geteuid()); + ASSERT_TRUE(FindByRemoteAddr(entries, &e, sockets->first_addr())) + << DescribeSecondInetSocket(*sockets); + EXPECT_EQ(e.uid, geteuid()); +} + +TEST(ProcNetUDP, FindMutualEntries) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4UDPBidirectionalBindSocketPair(0).Create()); + std::vector<UDPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + + EXPECT_TRUE(FindByLocalAddr(entries, nullptr, sockets->first_addr())) + << DescribeFirstInetSocket(*sockets); + EXPECT_TRUE(FindByRemoteAddr(entries, nullptr, sockets->first_addr())) + << DescribeSecondInetSocket(*sockets); + + EXPECT_TRUE(FindByLocalAddr(entries, nullptr, sockets->second_addr())) + << DescribeSecondInetSocket(*sockets); + EXPECT_TRUE(FindByRemoteAddr(entries, nullptr, sockets->second_addr())) + << DescribeFirstInetSocket(*sockets); +} + +TEST(ProcNetUDP, EntriesRemovedOnClose) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4UDPBidirectionalBindSocketPair(0).Create()); + std::vector<UDPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + + EXPECT_TRUE(FindByLocalAddr(entries, nullptr, sockets->first_addr())) + << DescribeFirstInetSocket(*sockets); + EXPECT_TRUE(FindByLocalAddr(entries, nullptr, sockets->second_addr())) + << DescribeSecondInetSocket(*sockets); + + EXPECT_THAT(close(sockets->release_first_fd()), SyscallSucceeds()); + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + // First socket's entry should be gone, but the second socket's entry should + // still exist. + EXPECT_FALSE(FindByLocalAddr(entries, nullptr, sockets->first_addr())) + << DescribeFirstInetSocket(*sockets); + EXPECT_TRUE(FindByLocalAddr(entries, nullptr, sockets->second_addr())) + << DescribeSecondInetSocket(*sockets); + + EXPECT_THAT(close(sockets->release_second_fd()), SyscallSucceeds()); + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + // Both entries should be gone. + EXPECT_FALSE(FindByLocalAddr(entries, nullptr, sockets->first_addr())) + << DescribeFirstInetSocket(*sockets); + EXPECT_FALSE(FindByLocalAddr(entries, nullptr, sockets->second_addr())) + << DescribeSecondInetSocket(*sockets); +} + +PosixErrorOr<std::unique_ptr<FileDescriptor>> BoundUDPSocket() { + ASSIGN_OR_RETURN_ERRNO(std::unique_ptr<FileDescriptor> socket, + IPv4UDPUnboundSocket(0).Create()); + struct sockaddr_in addr; + addr.sin_family = AF_INET; + addr.sin_addr.s_addr = htonl(INADDR_ANY); + addr.sin_port = 0; + + int res = bind(socket->get(), reinterpret_cast<const struct sockaddr*>(&addr), + sizeof(addr)); + if (res) { + return PosixError(errno, "bind()"); + } + return socket; +} + +TEST(ProcNetUDP, BoundEntry) { + std::unique_ptr<FileDescriptor> socket = + ASSERT_NO_ERRNO_AND_VALUE(BoundUDPSocket()); + struct sockaddr addr; + socklen_t len = sizeof(addr); + ASSERT_THAT(getsockname(socket->get(), &addr, &len), SyscallSucceeds()); + uint16_t port = PortFromInetSockaddr(&addr); + + std::vector<UDPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + UDPEntry e; + ASSERT_TRUE(ASSERT_NO_ERRNO_AND_VALUE(FindByFD(entries, &e, socket->get()))); + EXPECT_EQ(e.local_port, port); + EXPECT_EQ(e.remote_addr, 0); + EXPECT_EQ(e.remote_port, 0); +} + +TEST(ProcNetUDP, BoundSocketStateClosed) { + std::unique_ptr<FileDescriptor> socket = + ASSERT_NO_ERRNO_AND_VALUE(BoundUDPSocket()); + std::vector<UDPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + UDPEntry e; + ASSERT_TRUE(ASSERT_NO_ERRNO_AND_VALUE(FindByFD(entries, &e, socket->get()))); + EXPECT_EQ(e.state, TCP_CLOSE); +} + +TEST(ProcNetUDP, ConnectedSocketStateEstablished) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4UDPBidirectionalBindSocketPair(0).Create()); + std::vector<UDPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetUDPEntries()); + + UDPEntry e; + ASSERT_TRUE(FindByLocalAddr(entries, &e, sockets->first_addr())) + << DescribeFirstInetSocket(*sockets); + EXPECT_EQ(e.state, TCP_ESTABLISHED); + + ASSERT_TRUE(FindByLocalAddr(entries, &e, sockets->second_addr())) + << DescribeSecondInetSocket(*sockets); + EXPECT_EQ(e.state, TCP_ESTABLISHED); +} + +} // namespace +} // namespace testing +} // namespace gvisor diff --git a/test/util/fs_util.cc b/test/util/fs_util.cc index ae49725a0..f7d231b14 100644 --- a/test/util/fs_util.cc +++ b/test/util/fs_util.cc @@ -105,6 +105,15 @@ PosixErrorOr<struct stat> Stat(absl::string_view path) { return stat_buf; } +PosixErrorOr<struct stat> Fstat(int fd) { + struct stat stat_buf; + int res = fstat(fd, &stat_buf); + if (res < 0) { + return PosixError(errno, absl::StrCat("fstat ", fd)); + } + return stat_buf; +} + PosixErrorOr<bool> Exists(absl::string_view path) { struct stat stat_buf; int res = stat(std::string(path).c_str(), &stat_buf); diff --git a/test/util/fs_util.h b/test/util/fs_util.h index 3969f8309..e5b555891 100644 --- a/test/util/fs_util.h +++ b/test/util/fs_util.h @@ -35,6 +35,9 @@ PosixErrorOr<bool> Exists(absl::string_view path); // Returns a stat structure for the given path or an error. PosixErrorOr<struct stat> Stat(absl::string_view path); +// Returns a stat struct for the given fd. +PosixErrorOr<struct stat> Fstat(int fd); + // Deletes the file or directory at path or returns an error. PosixError Delete(absl::string_view path); |