diff options
-rw-r--r-- | pkg/sentry/fs/proc/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/fs/proc/net.go | 169 | ||||
-rw-r--r-- | test/syscalls/linux/BUILD | 15 | ||||
-rw-r--r-- | test/syscalls/linux/proc_net_tcp.cc | 281 | ||||
-rw-r--r-- | test/syscalls/linux/proc_net_unix.cc | 5 |
5 files changed, 457 insertions, 14 deletions
diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index b70c583f3..da41a10ab 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -31,6 +31,7 @@ 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 caa1a5c4d..37694620c 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -20,6 +20,7 @@ import ( "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" @@ -55,9 +56,8 @@ func (p *proc) newNetDir(ctx context.Context, k *kernel.Kernel, msrc *fs.MountSo "psched": newStaticProcInode(ctx, msrc, []byte(fmt.Sprintf("%08x %08x %08x %08x\n", uint64(time.Microsecond/time.Nanosecond), 64, 1000000, uint64(time.Second/time.Nanosecond)))), "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": newStaticProcInode(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode")), - - "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")), + "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), } @@ -210,10 +210,6 @@ func (n *netUnix) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]s } var buf bytes.Buffer - // Header - fmt.Fprintf(&buf, "Num RefCount Protocol Flags Type St Inode Path\n") - - // Entries for _, se := range n.k.ListSockets() { s := se.Sock.Get() if s == nil { @@ -222,6 +218,7 @@ func (n *netUnix) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]s } sfile := s.(*fs.File) if family, _, _ := sfile.FileOperations.(socket.Socket).Type(); family != linux.AF_UNIX { + s.DecRef() // Not a unix socket. continue } @@ -281,12 +278,160 @@ func (n *netUnix) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]s } fmt.Fprintf(&buf, "\n") - sfile.DecRef() + s.DecRef() + } + + data := []seqfile.SeqData{ + { + Buf: []byte("Num RefCount Protocol Flags Type St Inode Path\n"), + Handle: n, + }, + { + Buf: buf.Bytes(), + Handle: n, + }, + } + return data, 0 +} + +// netTCP implements seqfile.SeqSource for /proc/net/tcp. +// +// +stateify savable +type netTCP struct { + k *kernel.Kernel +} + +// NeedsUpdate implements seqfile.SeqSource.NeedsUpdate. +func (*netTCP) NeedsUpdate(generation int64) bool { + return true +} + +// ReadSeqFileData implements seqfile.SeqSource.ReadSeqFileData. +func (n *netTCP) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]seqfile.SeqData, int64) { + 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 %+v in socket table, racing with destruction?", se.Sock) + 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_STREAM) { + s.DecRef() + // Not tcp4 sockets. + continue + } + + // Linux's documentation for the fields below can be found at + // https://www.kernel.org/doc/Documentation/networking/proc_net_tcp.txt. + // For Linux's implementation, see net/ipv4/tcp_ipv4.c:get_tcp4_sock(). + // Note that the header doesn't contain labels for all the fields. + + // 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) + } + binary.LittleEndian.PutUint16(portBuf, localAddr.Port) + fmt.Fprintf(&buf, "%08X:%04X ", + binary.LittleEndian.Uint32(localAddr.Addr[:]), + portBuf) + + // Field: rem_address. + var remoteAddr linux.SockAddrInet + 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) + + // 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; timer active state and number of jiffies + // until timer expires. Unimplemented. + fmt.Fprintf(&buf, "%02X:%08X ", 0, 0) + + // Field: retrnsmt; number of unrecovered RTO timeouts. + // Unimplemented. + 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 { + fmt.Fprintf(&buf, "%5d ", uint32(uattr.Owner.UID.In(t.UserNamespace()).OrOverflow())) + } + + // Field: timeout; number of unanswered 0-window probes. + // Unimplemented. + fmt.Fprintf(&buf, "%8d ", 0) + + // Field: inode. + fmt.Fprintf(&buf, "%8d ", sfile.InodeID()) + + // Field: refcount. 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: retransmit timeout. Unimplemented. + fmt.Fprintf(&buf, "%d ", 0) + + // Field: predicted tick of soft clock (delayed ACK control data). + // Unimplemented. + fmt.Fprintf(&buf, "%d ", 0) + + // Field: (ack.quick<<1)|ack.pingpong, Unimplemented. + fmt.Fprintf(&buf, "%d ", 0) + + // Field: sending congestion window, Unimplemented. + fmt.Fprintf(&buf, "%d ", 0) + + // Field: Slow start size threshold, -1 if threshold >= 0xFFFF. + // Unimplemented, report as large threshold. + fmt.Fprintf(&buf, "%d", -1) + + fmt.Fprintf(&buf, "\n") + + s.DecRef() } - data := []seqfile.SeqData{{ - Buf: buf.Bytes(), - Handle: (*netUnix)(nil), - }} + data := []seqfile.SeqData{ + { + Buf: []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode \n"), + Handle: n, + }, + { + Buf: buf.Bytes(), + Handle: n, + }, + } return data, 0 } diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index 0618fea58..bfe3dfe04 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -3340,3 +3340,18 @@ cc_binary( "@com_google_googletest//:gtest", ], ) + +cc_binary( + name = "proc_net_tcp_test", + testonly = 1, + srcs = ["proc_net_tcp.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/proc_net_tcp.cc b/test/syscalls/linux/proc_net_tcp.cc new file mode 100644 index 000000000..578b20680 --- /dev/null +++ b/test/syscalls/linux/proc_net_tcp.cc @@ -0,0 +1,281 @@ +// 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::StrSplit; + +constexpr char kProcNetTCPHeader[] = + " sl local_address rem_address st tx_queue rx_queue tr tm->when " + "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; + uint16_t local_port; + + uint32_t remote_addr; + uint16_t remote_port; + + uint64_t state; + uint64_t uid; + 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, + std::function<bool(const TCPEntry&)> predicate) { + for (int i = 0; i < entries.size(); ++i) { + if (predicate(entries[i])) { + *match = entries[i]; + return true; + } + } + return false; +} + +bool FindByLocalAddr(std::vector<TCPEntry> entries, TCPEntry* match, + const struct sockaddr* addr) { + uint32_t host = IP(addr); + uint16_t port = Port(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, + const struct sockaddr* addr) { + uint32_t host = IP(addr); + uint16_t port = Port(addr); + return FindBy(entries, match, [host, port](const TCPEntry& e) { + return (e.remote_addr == host && e.remote_port == port); + }); +} + +// Returns a parsed representation of /proc/net/tcp entries. +PosixErrorOr<std::vector<TCPEntry>> ProcNetTCPEntries() { + std::string content; + RETURN_IF_ERRNO(GetContents("/proc/net/tcp", &content)); + + bool found_header = false; + 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) { + std::cerr << line << std::endl; + + if (!found_header) { + EXPECT_EQ(line, kProcNetTCPHeader); + found_header = true; + continue; + } + if (line.empty()) { + continue; + } + + // Parse a single entry from /proc/net/tcp. + // + // Example entries: + // + // clang-format off + // + // sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode + // 0: 00000000:006F 00000000:0000 0A 00000000:00000000 00:00000000 00000000 0 0 1968 1 0000000000000000 100 0 0 10 0 + // 1: 0100007F:7533 00000000:0000 0A 00000000:00000000 00:00000000 00000000 120 0 10684 1 0000000000000000 100 0 0 10 0 + // ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ + // 0 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 + // + // clang-format on + + TCPEntry 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])); + + entries.push_back(entry); + } + std::cerr << "<end of /proc/net/tcp>" << std::endl; + + return entries; +} + +TEST(ProcNetTCP, Exists) { + const std::string content = + ASSERT_NO_ERRNO_AND_VALUE(GetContents("/proc/net/tcp")); + const std::string header_line = StrCat(kProcNetTCPHeader, "\n"); + if (IsRunningOnGvisor()) { + // Should be just the header since we don't have any tcp sockets yet. + EXPECT_EQ(content, header_line); + } else { + // On a general linux machine, we could have abitrary sockets on the system, + // so just check the header. + EXPECT_THAT(content, ::testing::StartsWith(header_line)); + } +} + +TEST(ProcNetTCP, EntryUID) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4TCPAcceptBindSocketPair(0).Create()); + std::vector<TCPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); + TCPEntry e; + EXPECT_TRUE(FindByLocalAddr(entries, &e, sockets->first_addr())); + EXPECT_EQ(e.uid, geteuid()); + EXPECT_TRUE(FindByRemoteAddr(entries, &e, sockets->first_addr())); + EXPECT_EQ(e.uid, geteuid()); +} + +TEST(ProcNetTCP, BindAcceptConnect) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4TCPAcceptBindSocketPair(0).Create()); + std::vector<TCPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); + // We can only make assertions about the total number of entries if we control + // the entire "machine". + if (IsRunningOnGvisor()) { + EXPECT_EQ(entries.size(), 2); + } + + TCPEntry e; + EXPECT_TRUE(FindByLocalAddr(entries, &e, sockets->first_addr())); + EXPECT_TRUE(FindByRemoteAddr(entries, &e, sockets->first_addr())); +} + +TEST(ProcNetTCP, InodeReasonable) { + auto sockets = + ASSERT_NO_ERRNO_AND_VALUE(IPv4TCPAcceptBindSocketPair(0).Create()); + std::vector<TCPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); + + TCPEntry accepted_entry; + ASSERT_TRUE(FindByLocalAddr(entries, &accepted_entry, sockets->first_addr())); + EXPECT_NE(accepted_entry.inode, 0); + + TCPEntry client_entry; + ASSERT_TRUE(FindByRemoteAddr(entries, &client_entry, sockets->first_addr())); + EXPECT_NE(client_entry.inode, 0); + EXPECT_NE(accepted_entry.inode, client_entry.inode); +} + +TEST(ProcNetTCP, State) { + std::unique_ptr<FileDescriptor> server = + ASSERT_NO_ERRNO_AND_VALUE(IPv4TCPUnboundSocket(0).Create()); + + auto test_addr = V4Loopback(); + ASSERT_THAT( + bind(server->get(), reinterpret_cast<struct sockaddr*>(&test_addr.addr), + test_addr.addr_len), + SyscallSucceeds()); + + struct sockaddr addr; + socklen_t addrlen = sizeof(struct sockaddr); + ASSERT_THAT(getsockname(server->get(), &addr, &addrlen), SyscallSucceeds()); + ASSERT_EQ(addrlen, sizeof(struct sockaddr)); + + ASSERT_THAT(listen(server->get(), 10), SyscallSucceeds()); + std::vector<TCPEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); + TCPEntry listen_entry; + ASSERT_TRUE(FindByLocalAddr(entries, &listen_entry, &addr)); + EXPECT_EQ(listen_entry.state, TCP_LISTEN); + + std::unique_ptr<FileDescriptor> client = + ASSERT_NO_ERRNO_AND_VALUE(IPv4TCPUnboundSocket(0).Create()); + ASSERT_THAT(connect(client->get(), &addr, addrlen), SyscallSucceeds()); + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); + ASSERT_TRUE(FindByLocalAddr(entries, &listen_entry, &addr)); + EXPECT_EQ(listen_entry.state, TCP_LISTEN); + TCPEntry client_entry; + ASSERT_TRUE(FindByRemoteAddr(entries, &client_entry, &addr)); + EXPECT_EQ(client_entry.state, TCP_ESTABLISHED); + + 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); + + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcNetTCPEntries()); + TCPEntry accepted_entry; + ASSERT_TRUE(FindBy(entries, &accepted_entry, + [client_entry, accepted_local_host, + accepted_local_port](const TCPEntry& e) { + return e.local_addr == accepted_local_host && + e.local_port == accepted_local_port && + e.remote_addr == client_entry.local_addr && + e.remote_port == client_entry.local_port; + })); + EXPECT_EQ(accepted_entry.state, TCP_ESTABLISHED); +} + +} // namespace +} // namespace testing +} // namespace gvisor diff --git a/test/syscalls/linux/proc_net_unix.cc b/test/syscalls/linux/proc_net_unix.cc index 82d325c17..74acbe92c 100644 --- a/test/syscalls/linux/proc_net_unix.cc +++ b/test/syscalls/linux/proc_net_unix.cc @@ -162,7 +162,7 @@ PosixErrorOr<std::vector<UnixEntry>> ProcNetUnixEntries() { // 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<UnixEntry> entries, UnixEntry* match, - std::function<bool(UnixEntry)> predicate) { + std::function<bool(const UnixEntry&)> predicate) { for (int i = 0; i < entries.size(); ++i) { if (predicate(entries[i])) { *match = entries[i]; @@ -174,7 +174,8 @@ bool FindBy(std::vector<UnixEntry> entries, UnixEntry* match, bool FindByPath(std::vector<UnixEntry> entries, UnixEntry* match, const std::string& path) { - return FindBy(entries, match, [path](UnixEntry e) { return e.path == path; }); + return FindBy(entries, match, + [path](const UnixEntry& e) { return e.path == path; }); } TEST(ProcNetUnix, Exists) { |