summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry
diff options
context:
space:
mode:
authorDean Deng <deandeng@google.com>2020-05-01 12:53:15 -0700
committergVisor bot <gvisor-bot@google.com>2020-05-01 12:54:41 -0700
commit82bae30ceea0984c94af3085866b58ec9e69ea67 (patch)
tree8ed93228f9a3d6ec0d5333dc7f6649570dffe427 /pkg/sentry
parent8962b7840f577b15d526e360077eefdb5c6373ce (diff)
Port netstack, hostinet, and netlink sockets to VFS2.
All three follow the same pattern: 1. Refactor VFS1 sockets into socketOpsCommon, so that most of the methods can be shared with VFS2. 2. Create a FileDescriptionImpl with the corresponding socket operations, rewriting the few that cannot be shared with VFS1. 3. Set up a VFS2 socket provider that creates a socket by setting up a dentry in the global Kernel.socketMount and connecting it with a new FileDescription. This mostly completes the work for porting sockets to VFS2, and many syscall tests can be enabled as a result. There are several networking-related syscall tests that are still not passing: 1. net gofer tests 2. socketpair gofer tests 2. sendfile tests (splice is not implemented in VFS2 yet) Updates #1478, #1484, #1485 PiperOrigin-RevId: 309457331
Diffstat (limited to 'pkg/sentry')
-rw-r--r--pkg/sentry/fsimpl/gofer/socket.go9
-rw-r--r--pkg/sentry/fsimpl/host/host.go2
-rw-r--r--pkg/sentry/fsimpl/host/socket.go53
-rw-r--r--pkg/sentry/fsimpl/sockfs/sockfs.go5
-rw-r--r--pkg/sentry/socket/hostinet/BUILD5
-rw-r--r--pkg/sentry/socket/hostinet/socket.go105
-rw-r--r--pkg/sentry/socket/hostinet/socket_unsafe.go10
-rw-r--r--pkg/sentry/socket/hostinet/socket_vfs2.go183
-rw-r--r--pkg/sentry/socket/netlink/BUILD5
-rw-r--r--pkg/sentry/socket/netlink/provider.go5
-rw-r--r--pkg/sentry/socket/netlink/provider_vfs2.go71
-rw-r--r--pkg/sentry/socket/netlink/socket.go72
-rw-r--r--pkg/sentry/socket/netlink/socket_vfs2.go138
-rw-r--r--pkg/sentry/socket/netstack/BUILD5
-rw-r--r--pkg/sentry/socket/netstack/netstack.go72
-rw-r--r--pkg/sentry/socket/netstack/netstack_vfs2.go327
-rw-r--r--pkg/sentry/socket/netstack/provider.go4
-rw-r--r--pkg/sentry/socket/netstack/provider_vfs2.go141
-rw-r--r--pkg/sentry/socket/unix/unix_vfs2.go4
19 files changed, 1090 insertions, 126 deletions
diff --git a/pkg/sentry/fsimpl/gofer/socket.go b/pkg/sentry/fsimpl/gofer/socket.go
index 73835df91..d6dbe9092 100644
--- a/pkg/sentry/fsimpl/gofer/socket.go
+++ b/pkg/sentry/fsimpl/gofer/socket.go
@@ -87,7 +87,9 @@ func (e *endpoint) BidirectionalConnect(ctx context.Context, ce transport.Connec
returnConnect(c, c)
ce.Unlock()
- c.Init()
+ if err := c.Init(); err != nil {
+ return syserr.FromError(err)
+ }
return nil
}
@@ -99,7 +101,10 @@ func (e *endpoint) UnidirectionalConnect(ctx context.Context) (transport.Connect
if err != nil {
return nil, err
}
- c.Init()
+
+ if err := c.Init(); err != nil {
+ return nil, syserr.FromError(err)
+ }
// We don't need the receiver.
c.CloseRecv()
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index 33bd205af..34fbc69af 100644
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -411,7 +411,7 @@ func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount) (*vfs.F
return nil, syserror.ENOTTY
}
- ep, err := newEndpoint(ctx, i.hostFD)
+ ep, err := newEndpoint(ctx, i.hostFD, &i.queue)
if err != nil {
return nil, err
}
diff --git a/pkg/sentry/fsimpl/host/socket.go b/pkg/sentry/fsimpl/host/socket.go
index 39dd624a0..38f1fbfba 100644
--- a/pkg/sentry/fsimpl/host/socket.go
+++ b/pkg/sentry/fsimpl/host/socket.go
@@ -34,17 +34,16 @@ import (
"gvisor.dev/gvisor/pkg/waiter"
)
-// Create a new host-backed endpoint from the given fd.
-func newEndpoint(ctx context.Context, hostFD int) (transport.Endpoint, error) {
+// Create a new host-backed endpoint from the given fd and its corresponding
+// notification queue.
+func newEndpoint(ctx context.Context, hostFD int, queue *waiter.Queue) (transport.Endpoint, error) {
// Set up an external transport.Endpoint using the host fd.
addr := fmt.Sprintf("hostfd:[%d]", hostFD)
- var q waiter.Queue
- e, err := NewConnectedEndpoint(ctx, hostFD, &q, addr, true /* saveable */)
+ e, err := NewConnectedEndpoint(ctx, hostFD, addr, true /* saveable */)
if err != nil {
return nil, err.ToError()
}
- e.Init()
- ep := transport.NewExternal(ctx, e.stype, uniqueid.GlobalProviderFromContext(ctx), &q, e, e)
+ ep := transport.NewExternal(ctx, e.stype, uniqueid.GlobalProviderFromContext(ctx), queue, e, e)
return ep, nil
}
@@ -77,8 +76,6 @@ type ConnectedEndpoint struct {
// addr is the address at which this endpoint is bound.
addr string
- queue *waiter.Queue
-
// sndbuf is the size of the send buffer.
//
// N.B. When this is smaller than the host size, we present it via
@@ -134,11 +131,10 @@ func (c *ConnectedEndpoint) init() *syserr.Error {
// The caller is responsible for calling Init(). Additionaly, Release needs to
// be called twice because ConnectedEndpoint is both a transport.Receiver and
// transport.ConnectedEndpoint.
-func NewConnectedEndpoint(ctx context.Context, hostFD int, queue *waiter.Queue, addr string, saveable bool) (*ConnectedEndpoint, *syserr.Error) {
+func NewConnectedEndpoint(ctx context.Context, hostFD int, addr string, saveable bool) (*ConnectedEndpoint, *syserr.Error) {
e := ConnectedEndpoint{
- fd: hostFD,
- addr: addr,
- queue: queue,
+ fd: hostFD,
+ addr: addr,
}
if err := e.init(); err != nil {
@@ -151,13 +147,6 @@ func NewConnectedEndpoint(ctx context.Context, hostFD int, queue *waiter.Queue,
return &e, nil
}
-// Init will do the initialization required without holding other locks.
-func (c *ConnectedEndpoint) Init() {
- if err := fdnotifier.AddFD(int32(c.fd), c.queue); err != nil {
- panic(err)
- }
-}
-
// Send implements transport.ConnectedEndpoint.Send.
func (c *ConnectedEndpoint) Send(data [][]byte, controlMessages transport.ControlMessages, from tcpip.FullAddress) (int64, bool, *syserr.Error) {
c.mu.RLock()
@@ -332,7 +321,6 @@ func (c *ConnectedEndpoint) RecvMaxQueueSize() int64 {
}
func (c *ConnectedEndpoint) destroyLocked() {
- fdnotifier.RemoveFD(int32(c.fd))
c.fd = -1
}
@@ -350,14 +338,20 @@ func (c *ConnectedEndpoint) Release() {
func (c *ConnectedEndpoint) CloseUnread() {}
// SCMConnectedEndpoint represents an endpoint backed by a host fd that was
-// passed through a gofer Unix socket. It is almost the same as
-// ConnectedEndpoint, with the following differences:
+// passed through a gofer Unix socket. It resembles ConnectedEndpoint, with the
+// following differences:
// - SCMConnectedEndpoint is not saveable, because the host cannot guarantee
// the same descriptor number across S/R.
-// - SCMConnectedEndpoint holds ownership of its fd and is responsible for
-// closing it.
+// - SCMConnectedEndpoint holds ownership of its fd and notification queue.
type SCMConnectedEndpoint struct {
ConnectedEndpoint
+
+ queue *waiter.Queue
+}
+
+// Init will do the initialization required without holding other locks.
+func (e *SCMConnectedEndpoint) Init() error {
+ return fdnotifier.AddFD(int32(e.fd), e.queue)
}
// Release implements transport.ConnectedEndpoint.Release and
@@ -368,6 +362,7 @@ func (e *SCMConnectedEndpoint) Release() {
if err := syscall.Close(e.fd); err != nil {
log.Warningf("Failed to close host fd %d: %v", err)
}
+ fdnotifier.RemoveFD(int32(e.fd))
e.destroyLocked()
e.mu.Unlock()
})
@@ -380,11 +375,13 @@ func (e *SCMConnectedEndpoint) Release() {
// be called twice because ConnectedEndpoint is both a transport.Receiver and
// transport.ConnectedEndpoint.
func NewSCMEndpoint(ctx context.Context, hostFD int, queue *waiter.Queue, addr string) (*SCMConnectedEndpoint, *syserr.Error) {
- e := SCMConnectedEndpoint{ConnectedEndpoint{
- fd: hostFD,
- addr: addr,
+ e := SCMConnectedEndpoint{
+ ConnectedEndpoint: ConnectedEndpoint{
+ fd: hostFD,
+ addr: addr,
+ },
queue: queue,
- }}
+ }
if err := e.init(); err != nil {
return nil, err
diff --git a/pkg/sentry/fsimpl/sockfs/sockfs.go b/pkg/sentry/fsimpl/sockfs/sockfs.go
index 271134af8..dac2389fc 100644
--- a/pkg/sentry/fsimpl/sockfs/sockfs.go
+++ b/pkg/sentry/fsimpl/sockfs/sockfs.go
@@ -74,6 +74,11 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr
}
// NewDentry constructs and returns a sockfs dentry.
+//
+// TODO(gvisor.dev/issue/1476): Currently, we are using
+// sockfs.filesystem.NextIno() to get inode numbers. We should use
+// device-specific numbers, so that we are not using the same generator for
+// netstack, unix, etc.
func NewDentry(creds *auth.Credentials, ino uint64) *vfs.Dentry {
// File mode matches net/socket.c:sock_alloc.
filemode := linux.FileMode(linux.S_IFSOCK | 0600)
diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD
index 023bad156..deedd35f7 100644
--- a/pkg/sentry/socket/hostinet/BUILD
+++ b/pkg/sentry/socket/hostinet/BUILD
@@ -10,6 +10,7 @@ go_library(
"save_restore.go",
"socket.go",
"socket_unsafe.go",
+ "socket_vfs2.go",
"sockopt_impl.go",
"stack.go",
],
@@ -25,11 +26,15 @@ go_library(
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/fsimpl/kernfs",
+ "//pkg/sentry/fsimpl/sockfs",
+ "//pkg/sentry/hostfd",
"//pkg/sentry/inet",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/time",
"//pkg/sentry/socket",
"//pkg/sentry/socket/control",
+ "//pkg/sentry/vfs",
"//pkg/syserr",
"//pkg/syserror",
"//pkg/tcpip/stack",
diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go
index 22f78d2e2..b49433326 100644
--- a/pkg/sentry/socket/hostinet/socket.go
+++ b/pkg/sentry/socket/hostinet/socket.go
@@ -25,6 +25,7 @@ import (
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/safemem"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -49,6 +50,8 @@ const (
maxControlLen = 1024
)
+// LINT.IfChange
+
// socketOperations implements fs.FileOperations and socket.Socket for a socket
// implemented using a host socket.
type socketOperations struct {
@@ -59,23 +62,37 @@ type socketOperations struct {
fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+
+ socketOpsCommon
+}
+
+// socketOpsCommon contains the socket operations common to VFS1 and VFS2.
+//
+// +stateify savable
+type socketOpsCommon struct {
socket.SendReceiveTimeout
family int // Read-only.
stype linux.SockType // Read-only.
protocol int // Read-only.
- fd int // must be O_NONBLOCK
queue waiter.Queue
+
+ // fd is the host socket fd. It must have O_NONBLOCK, so that operations
+ // will return EWOULDBLOCK instead of blocking on the host. This allows us to
+ // handle blocking behavior independently in the sentry.
+ fd int
}
var _ = socket.Socket(&socketOperations{})
func newSocketFile(ctx context.Context, family int, stype linux.SockType, protocol int, fd int, nonblock bool) (*fs.File, *syserr.Error) {
s := &socketOperations{
- family: family,
- stype: stype,
- protocol: protocol,
- fd: fd,
+ socketOpsCommon: socketOpsCommon{
+ family: family,
+ stype: stype,
+ protocol: protocol,
+ fd: fd,
+ },
}
if err := fdnotifier.AddFD(int32(fd), &s.queue); err != nil {
return nil, syserr.FromError(err)
@@ -86,28 +103,33 @@ func newSocketFile(ctx context.Context, family int, stype linux.SockType, protoc
}
// Release implements fs.FileOperations.Release.
-func (s *socketOperations) Release() {
+func (s *socketOpsCommon) Release() {
fdnotifier.RemoveFD(int32(s.fd))
syscall.Close(s.fd)
}
// Readiness implements waiter.Waitable.Readiness.
-func (s *socketOperations) Readiness(mask waiter.EventMask) waiter.EventMask {
+func (s *socketOpsCommon) Readiness(mask waiter.EventMask) waiter.EventMask {
return fdnotifier.NonBlockingPoll(int32(s.fd), mask)
}
// EventRegister implements waiter.Waitable.EventRegister.
-func (s *socketOperations) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+func (s *socketOpsCommon) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
s.queue.EventRegister(e, mask)
fdnotifier.UpdateFD(int32(s.fd))
}
// EventUnregister implements waiter.Waitable.EventUnregister.
-func (s *socketOperations) EventUnregister(e *waiter.Entry) {
+func (s *socketOpsCommon) EventUnregister(e *waiter.Entry) {
s.queue.EventUnregister(e)
fdnotifier.UpdateFD(int32(s.fd))
}
+// Ioctl implements fs.FileOperations.Ioctl.
+func (s *socketOperations) Ioctl(ctx context.Context, _ *fs.File, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ return ioctl(ctx, s.fd, io, args)
+}
+
// Read implements fs.FileOperations.Read.
func (s *socketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ int64) (int64, error) {
n, err := dst.CopyOutFrom(ctx, safemem.ReaderFunc(func(dsts safemem.BlockSeq) (uint64, error) {
@@ -155,7 +177,7 @@ func (s *socketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO
}
// Connect implements socket.Socket.Connect.
-func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
+func (s *socketOpsCommon) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
if len(sockaddr) > sizeofSockaddr {
sockaddr = sockaddr[:sizeofSockaddr]
}
@@ -195,7 +217,7 @@ func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking boo
}
// Accept implements socket.Socket.Accept.
-func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (int32, linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (int32, linux.SockAddr, uint32, *syserr.Error) {
var peerAddr linux.SockAddr
var peerAddrBuf []byte
var peerAddrlen uint32
@@ -209,7 +231,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int,
}
// Conservatively ignore all flags specified by the application and add
- // SOCK_NONBLOCK since socketOperations requires it.
+ // SOCK_NONBLOCK since socketOpsCommon requires it.
fd, syscallErr := accept4(s.fd, peerAddrPtr, peerAddrlenPtr, syscall.SOCK_NONBLOCK|syscall.SOCK_CLOEXEC)
if blocking {
var ch chan struct{}
@@ -235,23 +257,41 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int,
return 0, peerAddr, peerAddrlen, syserr.FromError(syscallErr)
}
- f, err := newSocketFile(t, s.family, s.stype, s.protocol, fd, flags&syscall.SOCK_NONBLOCK != 0)
- if err != nil {
- syscall.Close(fd)
- return 0, nil, 0, err
- }
- defer f.DecRef()
+ var (
+ kfd int32
+ kerr error
+ )
+ if kernel.VFS2Enabled {
+ f, err := newVFS2Socket(t, s.family, s.stype, s.protocol, fd, uint32(flags&syscall.SOCK_NONBLOCK))
+ if err != nil {
+ syscall.Close(fd)
+ return 0, nil, 0, err
+ }
+ defer f.DecRef()
- kfd, kerr := t.NewFDFrom(0, f, kernel.FDFlags{
- CloseOnExec: flags&syscall.SOCK_CLOEXEC != 0,
- })
- t.Kernel().RecordSocket(f)
+ kfd, kerr = t.NewFDFromVFS2(0, f, kernel.FDFlags{
+ CloseOnExec: flags&syscall.SOCK_CLOEXEC != 0,
+ })
+ t.Kernel().RecordSocketVFS2(f)
+ } else {
+ f, err := newSocketFile(t, s.family, s.stype, s.protocol, fd, flags&syscall.SOCK_NONBLOCK != 0)
+ if err != nil {
+ syscall.Close(fd)
+ return 0, nil, 0, err
+ }
+ defer f.DecRef()
+
+ kfd, kerr = t.NewFDFrom(0, f, kernel.FDFlags{
+ CloseOnExec: flags&syscall.SOCK_CLOEXEC != 0,
+ })
+ t.Kernel().RecordSocket(f)
+ }
return kfd, peerAddr, peerAddrlen, syserr.FromError(kerr)
}
// Bind implements socket.Socket.Bind.
-func (s *socketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
+func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
if len(sockaddr) > sizeofSockaddr {
sockaddr = sockaddr[:sizeofSockaddr]
}
@@ -264,12 +304,12 @@ func (s *socketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
}
// Listen implements socket.Socket.Listen.
-func (s *socketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error {
+func (s *socketOpsCommon) Listen(t *kernel.Task, backlog int) *syserr.Error {
return syserr.FromError(syscall.Listen(s.fd, backlog))
}
// Shutdown implements socket.Socket.Shutdown.
-func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error {
+func (s *socketOpsCommon) Shutdown(t *kernel.Task, how int) *syserr.Error {
switch how {
case syscall.SHUT_RD, syscall.SHUT_WR, syscall.SHUT_RDWR:
return syserr.FromError(syscall.Shutdown(s.fd, how))
@@ -279,7 +319,7 @@ func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error {
}
// GetSockOpt implements socket.Socket.GetSockOpt.
-func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outPtr usermem.Addr, outLen int) (interface{}, *syserr.Error) {
+func (s *socketOpsCommon) GetSockOpt(t *kernel.Task, level int, name int, outPtr usermem.Addr, outLen int) (interface{}, *syserr.Error) {
if outLen < 0 {
return nil, syserr.ErrInvalidArgument
}
@@ -328,7 +368,7 @@ func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outPt
}
// SetSockOpt implements socket.Socket.SetSockOpt.
-func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error {
+func (s *socketOpsCommon) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error {
// Whitelist options and constrain option length.
optlen := setSockOptLen(t, level, name)
switch level {
@@ -374,7 +414,7 @@ func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt [
}
// RecvMsg implements socket.Socket.RecvMsg.
-func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlLen uint64) (int, int, linux.SockAddr, uint32, socket.ControlMessages, *syserr.Error) {
+func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlLen uint64) (int, int, linux.SockAddr, uint32, socket.ControlMessages, *syserr.Error) {
// Whitelist flags.
//
// FIXME(jamieliu): We can't support MSG_ERRQUEUE because it uses ancillary
@@ -496,7 +536,7 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags
}
// SendMsg implements socket.Socket.SendMsg.
-func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) {
+func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) {
// Whitelist flags.
if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 {
return 0, syserr.ErrInvalidArgument
@@ -585,7 +625,7 @@ func translateIOSyscallError(err error) error {
}
// State implements socket.Socket.State.
-func (s *socketOperations) State() uint32 {
+func (s *socketOpsCommon) State() uint32 {
info := linux.TCPInfo{}
buf, err := getsockopt(s.fd, syscall.SOL_TCP, syscall.TCP_INFO, linux.SizeOfTCPInfo)
if err != nil {
@@ -607,7 +647,7 @@ func (s *socketOperations) State() uint32 {
}
// Type implements socket.Socket.Type.
-func (s *socketOperations) Type() (family int, skType linux.SockType, protocol int) {
+func (s *socketOpsCommon) Type() (family int, skType linux.SockType, protocol int) {
return s.family, s.stype, s.protocol
}
@@ -663,8 +703,11 @@ func (p *socketProvider) Pair(t *kernel.Task, stype linux.SockType, protocol int
return nil, nil, nil
}
+// LINT.ThenChange(./socket_vfs2.go)
+
func init() {
for _, family := range []int{syscall.AF_INET, syscall.AF_INET6} {
socket.RegisterProvider(family, &socketProvider{family})
+ socket.RegisterProviderVFS2(family, &socketProviderVFS2{})
}
}
diff --git a/pkg/sentry/socket/hostinet/socket_unsafe.go b/pkg/sentry/socket/hostinet/socket_unsafe.go
index cd67234d2..3f420c2ec 100644
--- a/pkg/sentry/socket/hostinet/socket_unsafe.go
+++ b/pkg/sentry/socket/hostinet/socket_unsafe.go
@@ -21,7 +21,6 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/sentry/arch"
- "gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/socket"
"gvisor.dev/gvisor/pkg/syserr"
@@ -54,12 +53,11 @@ func writev(fd int, srcs []syscall.Iovec) (uint64, error) {
return uint64(n), nil
}
-// Ioctl implements fs.FileOperations.Ioctl.
-func (s *socketOperations) Ioctl(ctx context.Context, _ *fs.File, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+func ioctl(ctx context.Context, fd int, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
switch cmd := uintptr(args[1].Int()); cmd {
case syscall.TIOCINQ, syscall.TIOCOUTQ:
var val int32
- if _, _, errno := syscall.Syscall(syscall.SYS_IOCTL, uintptr(s.fd), cmd, uintptr(unsafe.Pointer(&val))); errno != 0 {
+ if _, _, errno := syscall.Syscall(syscall.SYS_IOCTL, uintptr(fd), cmd, uintptr(unsafe.Pointer(&val))); errno != 0 {
return 0, translateIOSyscallError(errno)
}
var buf [4]byte
@@ -93,7 +91,7 @@ func getsockopt(fd int, level, name int, optlen int) ([]byte, error) {
}
// GetSockName implements socket.Socket.GetSockName.
-func (s *socketOperations) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
addr := make([]byte, sizeofSockaddr)
addrlen := uint32(len(addr))
_, _, errno := syscall.Syscall(syscall.SYS_GETSOCKNAME, uintptr(s.fd), uintptr(unsafe.Pointer(&addr[0])), uintptr(unsafe.Pointer(&addrlen)))
@@ -104,7 +102,7 @@ func (s *socketOperations) GetSockName(t *kernel.Task) (linux.SockAddr, uint32,
}
// GetPeerName implements socket.Socket.GetPeerName.
-func (s *socketOperations) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
addr := make([]byte, sizeofSockaddr)
addrlen := uint32(len(addr))
_, _, errno := syscall.Syscall(syscall.SYS_GETPEERNAME, uintptr(s.fd), uintptr(unsafe.Pointer(&addr[0])), uintptr(unsafe.Pointer(&addrlen)))
diff --git a/pkg/sentry/socket/hostinet/socket_vfs2.go b/pkg/sentry/socket/hostinet/socket_vfs2.go
new file mode 100644
index 000000000..b03ca2f26
--- /dev/null
+++ b/pkg/sentry/socket/hostinet/socket_vfs2.go
@@ -0,0 +1,183 @@
+// Copyright 2020 The gVisor Authors.
+//
+// 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.
+
+package hostinet
+
+import (
+ "syscall"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fdnotifier"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/sockfs"
+ "gvisor.dev/gvisor/pkg/sentry/hostfd"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/socket"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserr"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+type socketVFS2 struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ // TODO(gvisor.dev/issue/1484): VFS1 stores internal metadata for hostinet.
+ // We should perhaps rely on the host, much like in hostfs.
+ vfs.DentryMetadataFileDescriptionImpl
+
+ socketOpsCommon
+}
+
+var _ = socket.SocketVFS2(&socketVFS2{})
+
+func newVFS2Socket(t *kernel.Task, family int, stype linux.SockType, protocol int, fd int, flags uint32) (*vfs.FileDescription, *syserr.Error) {
+ mnt := t.Kernel().SocketMount()
+ fs := mnt.Filesystem().Impl().(*kernfs.Filesystem)
+ d := sockfs.NewDentry(t.Credentials(), fs.NextIno())
+
+ s := &socketVFS2{
+ socketOpsCommon: socketOpsCommon{
+ family: family,
+ stype: stype,
+ protocol: protocol,
+ fd: fd,
+ },
+ }
+ if err := fdnotifier.AddFD(int32(fd), &s.queue); err != nil {
+ return nil, syserr.FromError(err)
+ }
+ vfsfd := &s.vfsfd
+ if err := vfsfd.Init(s, linux.O_RDWR|(flags&linux.O_NONBLOCK), mnt, d, &vfs.FileDescriptionOptions{
+ DenyPRead: true,
+ DenyPWrite: true,
+ UseDentryMetadata: true,
+ }); err != nil {
+ return nil, syserr.FromError(err)
+ }
+ return vfsfd, nil
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (s *socketVFS2) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return s.socketOpsCommon.Readiness(mask)
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (s *socketVFS2) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ s.socketOpsCommon.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (s *socketVFS2) EventUnregister(e *waiter.Entry) {
+ s.socketOpsCommon.EventUnregister(e)
+}
+
+// Ioctl implements vfs.FileDescriptionImpl.
+func (s *socketVFS2) Ioctl(ctx context.Context, uio usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ return ioctl(ctx, s.fd, uio, args)
+}
+
+// PRead implements vfs.FileDescriptionImpl.
+func (s *socketVFS2) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
+ return 0, syserror.ESPIPE
+}
+
+// Read implements vfs.FileDescriptionImpl.
+func (s *socketVFS2) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+ // All flags other than RWF_NOWAIT should be ignored.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
+ if opts.Flags != 0 {
+ return 0, syserror.EOPNOTSUPP
+ }
+
+ reader := hostfd.GetReadWriterAt(int32(s.fd), -1, opts.Flags)
+ n, err := dst.CopyOutFrom(ctx, reader)
+ hostfd.PutReadWriterAt(reader)
+ return int64(n), err
+}
+
+// PWrite implements vfs.FileDescriptionImpl.
+func (s *socketVFS2) PWrite(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
+ return 0, syserror.ESPIPE
+}
+
+// Write implements vfs.FileDescriptionImpl.
+func (s *socketVFS2) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+ // All flags other than RWF_NOWAIT should be ignored.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
+ if opts.Flags != 0 {
+ return 0, syserror.EOPNOTSUPP
+ }
+
+ writer := hostfd.GetReadWriterAt(int32(s.fd), -1, opts.Flags)
+ n, err := src.CopyInTo(ctx, writer)
+ hostfd.PutReadWriterAt(writer)
+ return int64(n), err
+}
+
+type socketProviderVFS2 struct {
+ family int
+}
+
+// Socket implements socket.ProviderVFS2.Socket.
+func (p *socketProviderVFS2) Socket(t *kernel.Task, stypeflags linux.SockType, protocol int) (*vfs.FileDescription, *syserr.Error) {
+ // Check that we are using the host network stack.
+ stack := t.NetworkContext()
+ if stack == nil {
+ return nil, nil
+ }
+ if _, ok := stack.(*Stack); !ok {
+ return nil, nil
+ }
+
+ // Only accept TCP and UDP.
+ stype := stypeflags & linux.SOCK_TYPE_MASK
+ switch stype {
+ case syscall.SOCK_STREAM:
+ switch protocol {
+ case 0, syscall.IPPROTO_TCP:
+ // ok
+ default:
+ return nil, nil
+ }
+ case syscall.SOCK_DGRAM:
+ switch protocol {
+ case 0, syscall.IPPROTO_UDP:
+ // ok
+ default:
+ return nil, nil
+ }
+ default:
+ return nil, nil
+ }
+
+ // Conservatively ignore all flags specified by the application and add
+ // SOCK_NONBLOCK since socketOperations requires it. Pass a protocol of 0
+ // to simplify the syscall filters, since 0 and IPPROTO_* are equivalent.
+ fd, err := syscall.Socket(p.family, int(stype)|syscall.SOCK_NONBLOCK|syscall.SOCK_CLOEXEC, 0)
+ if err != nil {
+ return nil, syserr.FromError(err)
+ }
+ return newVFS2Socket(t, p.family, stype, protocol, fd, uint32(stypeflags&syscall.SOCK_NONBLOCK))
+}
+
+// Pair implements socket.Provider.Pair.
+func (p *socketProviderVFS2) Pair(t *kernel.Task, stype linux.SockType, protocol int) (*vfs.FileDescription, *vfs.FileDescription, *syserr.Error) {
+ // Not supported by AF_INET/AF_INET6.
+ return nil, nil, nil
+}
diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD
index 1911cd9b8..09ca00a4a 100644
--- a/pkg/sentry/socket/netlink/BUILD
+++ b/pkg/sentry/socket/netlink/BUILD
@@ -7,7 +7,9 @@ go_library(
srcs = [
"message.go",
"provider.go",
+ "provider_vfs2.go",
"socket.go",
+ "socket_vfs2.go",
],
visibility = ["//pkg/sentry:internal"],
deps = [
@@ -18,6 +20,8 @@ go_library(
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/fsimpl/kernfs",
+ "//pkg/sentry/fsimpl/sockfs",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/time",
@@ -25,6 +29,7 @@ go_library(
"//pkg/sentry/socket/netlink/port",
"//pkg/sentry/socket/unix",
"//pkg/sentry/socket/unix/transport",
+ "//pkg/sentry/vfs",
"//pkg/sync",
"//pkg/syserr",
"//pkg/syserror",
diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go
index b0dc70e5c..0d45e5053 100644
--- a/pkg/sentry/socket/netlink/provider.go
+++ b/pkg/sentry/socket/netlink/provider.go
@@ -67,6 +67,8 @@ func RegisterProvider(protocol int, provider Provider) {
protocols[protocol] = provider
}
+// LINT.IfChange
+
// socketProvider implements socket.Provider.
type socketProvider struct {
}
@@ -105,7 +107,10 @@ func (*socketProvider) Pair(*kernel.Task, linux.SockType, int) (*fs.File, *fs.Fi
return nil, nil, syserr.ErrNotSupported
}
+// LINT.ThenChange(./provider_vfs2.go)
+
// init registers the socket provider.
func init() {
socket.RegisterProvider(linux.AF_NETLINK, &socketProvider{})
+ socket.RegisterProviderVFS2(linux.AF_NETLINK, &socketProviderVFS2{})
}
diff --git a/pkg/sentry/socket/netlink/provider_vfs2.go b/pkg/sentry/socket/netlink/provider_vfs2.go
new file mode 100644
index 000000000..dcd92b5cd
--- /dev/null
+++ b/pkg/sentry/socket/netlink/provider_vfs2.go
@@ -0,0 +1,71 @@
+// Copyright 2018 The gVisor Authors.
+//
+// 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.
+
+package netlink
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/sockfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserr"
+)
+
+// socketProviderVFS2 implements socket.Provider.
+type socketProviderVFS2 struct {
+}
+
+// Socket implements socket.Provider.Socket.
+func (*socketProviderVFS2) Socket(t *kernel.Task, stype linux.SockType, protocol int) (*vfs.FileDescription, *syserr.Error) {
+ // Netlink sockets must be specified as datagram or raw, but they
+ // behave the same regardless of type.
+ if stype != linux.SOCK_DGRAM && stype != linux.SOCK_RAW {
+ return nil, syserr.ErrSocketNotSupported
+ }
+
+ provider, ok := protocols[protocol]
+ if !ok {
+ return nil, syserr.ErrProtocolNotSupported
+ }
+
+ p, err := provider(t)
+ if err != nil {
+ return nil, err
+ }
+
+ s, err := NewVFS2(t, stype, p)
+ if err != nil {
+ return nil, err
+ }
+
+ vfsfd := &s.vfsfd
+ mnt := t.Kernel().SocketMount()
+ fs := mnt.Filesystem().Impl().(*kernfs.Filesystem)
+ d := sockfs.NewDentry(t.Credentials(), fs.NextIno())
+ if err := vfsfd.Init(s, linux.O_RDWR, mnt, d, &vfs.FileDescriptionOptions{
+ DenyPRead: true,
+ DenyPWrite: true,
+ UseDentryMetadata: true,
+ }); err != nil {
+ return nil, syserr.FromError(err)
+ }
+ return vfsfd, nil
+}
+
+// Pair implements socket.Provider.Pair by returning an error.
+func (*socketProviderVFS2) Pair(*kernel.Task, linux.SockType, int) (*vfs.FileDescription, *vfs.FileDescription, *syserr.Error) {
+ // Netlink sockets never supports creating socket pairs.
+ return nil, nil, syserr.ErrNotSupported
+}
diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go
index 2ca02567d..81f34c5a2 100644
--- a/pkg/sentry/socket/netlink/socket.go
+++ b/pkg/sentry/socket/netlink/socket.go
@@ -58,6 +58,8 @@ var errNoFilter = syserr.New("no filter attached", linux.ENOENT)
// netlinkSocketDevice is the netlink socket virtual device.
var netlinkSocketDevice = device.NewAnonDevice()
+// LINT.IfChange
+
// Socket is the base socket type for netlink sockets.
//
// This implementation only supports userspace sending and receiving messages
@@ -74,6 +76,14 @@ type Socket struct {
fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+
+ socketOpsCommon
+}
+
+// socketOpsCommon contains the socket operations common to VFS1 and VFS2.
+//
+// +stateify savable
+type socketOpsCommon struct {
socket.SendReceiveTimeout
// ports provides netlink port allocation.
@@ -140,17 +150,19 @@ func NewSocket(t *kernel.Task, skType linux.SockType, protocol Protocol) (*Socke
}
return &Socket{
- ports: t.Kernel().NetlinkPorts(),
- protocol: protocol,
- skType: skType,
- ep: ep,
- connection: connection,
- sendBufferSize: defaultSendBufferSize,
+ socketOpsCommon: socketOpsCommon{
+ ports: t.Kernel().NetlinkPorts(),
+ protocol: protocol,
+ skType: skType,
+ ep: ep,
+ connection: connection,
+ sendBufferSize: defaultSendBufferSize,
+ },
}, nil
}
// Release implements fs.FileOperations.Release.
-func (s *Socket) Release() {
+func (s *socketOpsCommon) Release() {
s.connection.Release()
s.ep.Close()
@@ -160,7 +172,7 @@ func (s *Socket) Release() {
}
// Readiness implements waiter.Waitable.Readiness.
-func (s *Socket) Readiness(mask waiter.EventMask) waiter.EventMask {
+func (s *socketOpsCommon) Readiness(mask waiter.EventMask) waiter.EventMask {
// ep holds messages to be read and thus handles EventIn readiness.
ready := s.ep.Readiness(mask)
@@ -174,18 +186,18 @@ func (s *Socket) Readiness(mask waiter.EventMask) waiter.EventMask {
}
// EventRegister implements waiter.Waitable.EventRegister.
-func (s *Socket) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+func (s *socketOpsCommon) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
s.ep.EventRegister(e, mask)
// Writable readiness never changes, so no registration is needed.
}
// EventUnregister implements waiter.Waitable.EventUnregister.
-func (s *Socket) EventUnregister(e *waiter.Entry) {
+func (s *socketOpsCommon) EventUnregister(e *waiter.Entry) {
s.ep.EventUnregister(e)
}
// Passcred implements transport.Credentialer.Passcred.
-func (s *Socket) Passcred() bool {
+func (s *socketOpsCommon) Passcred() bool {
s.mu.Lock()
passcred := s.passcred
s.mu.Unlock()
@@ -193,7 +205,7 @@ func (s *Socket) Passcred() bool {
}
// ConnectedPasscred implements transport.Credentialer.ConnectedPasscred.
-func (s *Socket) ConnectedPasscred() bool {
+func (s *socketOpsCommon) ConnectedPasscred() bool {
// This socket is connected to the kernel, which doesn't need creds.
//
// This is arbitrary, as ConnectedPasscred on this type has no callers.
@@ -227,7 +239,7 @@ func ExtractSockAddr(b []byte) (*linux.SockAddrNetlink, *syserr.Error) {
// port of 0 defaults to the ThreadGroup ID.
//
// Preconditions: mu is held.
-func (s *Socket) bindPort(t *kernel.Task, port int32) *syserr.Error {
+func (s *socketOpsCommon) bindPort(t *kernel.Task, port int32) *syserr.Error {
if s.bound {
// Re-binding is only allowed if the port doesn't change.
if port != s.portID {
@@ -251,7 +263,7 @@ func (s *Socket) bindPort(t *kernel.Task, port int32) *syserr.Error {
}
// Bind implements socket.Socket.Bind.
-func (s *Socket) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
+func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
a, err := ExtractSockAddr(sockaddr)
if err != nil {
return err
@@ -269,7 +281,7 @@ func (s *Socket) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
}
// Connect implements socket.Socket.Connect.
-func (s *Socket) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
+func (s *socketOpsCommon) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
a, err := ExtractSockAddr(sockaddr)
if err != nil {
return err
@@ -300,25 +312,25 @@ func (s *Socket) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr
}
// Accept implements socket.Socket.Accept.
-func (s *Socket) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (int32, linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (int32, linux.SockAddr, uint32, *syserr.Error) {
// Netlink sockets never support accept.
return 0, nil, 0, syserr.ErrNotSupported
}
// Listen implements socket.Socket.Listen.
-func (s *Socket) Listen(t *kernel.Task, backlog int) *syserr.Error {
+func (s *socketOpsCommon) Listen(t *kernel.Task, backlog int) *syserr.Error {
// Netlink sockets never support listen.
return syserr.ErrNotSupported
}
// Shutdown implements socket.Socket.Shutdown.
-func (s *Socket) Shutdown(t *kernel.Task, how int) *syserr.Error {
+func (s *socketOpsCommon) Shutdown(t *kernel.Task, how int) *syserr.Error {
// Netlink sockets never support shutdown.
return syserr.ErrNotSupported
}
// GetSockOpt implements socket.Socket.GetSockOpt.
-func (s *Socket) GetSockOpt(t *kernel.Task, level int, name int, outPtr usermem.Addr, outLen int) (interface{}, *syserr.Error) {
+func (s *socketOpsCommon) GetSockOpt(t *kernel.Task, level int, name int, outPtr usermem.Addr, outLen int) (interface{}, *syserr.Error) {
switch level {
case linux.SOL_SOCKET:
switch name {
@@ -369,7 +381,7 @@ func (s *Socket) GetSockOpt(t *kernel.Task, level int, name int, outPtr usermem.
}
// SetSockOpt implements socket.Socket.SetSockOpt.
-func (s *Socket) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error {
+func (s *socketOpsCommon) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error {
switch level {
case linux.SOL_SOCKET:
switch name {
@@ -466,7 +478,7 @@ func (s *Socket) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *sy
}
// GetSockName implements socket.Socket.GetSockName.
-func (s *Socket) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
s.mu.Lock()
defer s.mu.Unlock()
@@ -478,7 +490,7 @@ func (s *Socket) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Er
}
// GetPeerName implements socket.Socket.GetPeerName.
-func (s *Socket) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
sa := &linux.SockAddrNetlink{
Family: linux.AF_NETLINK,
// TODO(b/68878065): Support non-kernel peers. For now the peer
@@ -489,7 +501,7 @@ func (s *Socket) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Er
}
// RecvMsg implements socket.Socket.RecvMsg.
-func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, linux.SockAddr, uint32, socket.ControlMessages, *syserr.Error) {
+func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, linux.SockAddr, uint32, socket.ControlMessages, *syserr.Error) {
from := &linux.SockAddrNetlink{
Family: linux.AF_NETLINK,
PortID: 0,
@@ -590,7 +602,7 @@ func (kernelSCM) Credentials(*kernel.Task) (kernel.ThreadID, auth.UID, auth.GID)
var kernelCreds = &kernelSCM{}
// sendResponse sends the response messages in ms back to userspace.
-func (s *Socket) sendResponse(ctx context.Context, ms *MessageSet) *syserr.Error {
+func (s *socketOpsCommon) sendResponse(ctx context.Context, ms *MessageSet) *syserr.Error {
// Linux combines multiple netlink messages into a single datagram.
bufs := make([][]byte, 0, len(ms.Messages))
for _, m := range ms.Messages {
@@ -666,7 +678,7 @@ func dumpAckMesage(hdr linux.NetlinkMessageHeader, ms *MessageSet) {
// processMessages handles each message in buf, passing it to the protocol
// handler for final handling.
-func (s *Socket) processMessages(ctx context.Context, buf []byte) *syserr.Error {
+func (s *socketOpsCommon) processMessages(ctx context.Context, buf []byte) *syserr.Error {
for len(buf) > 0 {
msg, rest, ok := ParseMessage(buf)
if !ok {
@@ -698,7 +710,7 @@ func (s *Socket) processMessages(ctx context.Context, buf []byte) *syserr.Error
}
// sendMsg is the core of message send, used for SendMsg and Write.
-func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) {
+func (s *socketOpsCommon) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) {
dstPort := int32(0)
if len(to) != 0 {
@@ -745,7 +757,7 @@ func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte,
}
// SendMsg implements socket.Socket.SendMsg.
-func (s *Socket) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) {
+func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) {
return s.sendMsg(t, src, to, flags, controlMessages)
}
@@ -756,11 +768,13 @@ func (s *Socket) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence,
}
// State implements socket.Socket.State.
-func (s *Socket) State() uint32 {
+func (s *socketOpsCommon) State() uint32 {
return s.ep.State()
}
// Type implements socket.Socket.Type.
-func (s *Socket) Type() (family int, skType linux.SockType, protocol int) {
+func (s *socketOpsCommon) Type() (family int, skType linux.SockType, protocol int) {
return linux.AF_NETLINK, s.skType, s.protocol.Protocol()
}
+
+// LINT.ThenChange(./socket_vfs2.go)
diff --git a/pkg/sentry/socket/netlink/socket_vfs2.go b/pkg/sentry/socket/netlink/socket_vfs2.go
new file mode 100644
index 000000000..b854bf990
--- /dev/null
+++ b/pkg/sentry/socket/netlink/socket_vfs2.go
@@ -0,0 +1,138 @@
+// Copyright 2018 The gVisor Authors.
+//
+// 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.
+
+package netlink
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "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/vfs"
+ "gvisor.dev/gvisor/pkg/syserr"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/tcpip"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// SocketVFS2 is the base VFS2 socket type for netlink sockets.
+//
+// This implementation only supports userspace sending and receiving messages
+// to/from the kernel.
+//
+// SocketVFS2 implements socket.SocketVFS2 and transport.Credentialer.
+type SocketVFS2 struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ socketOpsCommon
+}
+
+var _ socket.SocketVFS2 = (*SocketVFS2)(nil)
+var _ transport.Credentialer = (*SocketVFS2)(nil)
+
+// NewVFS2 creates a new SocketVFS2.
+func NewVFS2(t *kernel.Task, skType linux.SockType, protocol Protocol) (*SocketVFS2, *syserr.Error) {
+ // Datagram endpoint used to buffer kernel -> user messages.
+ ep := transport.NewConnectionless(t)
+
+ // Bind the endpoint for good measure so we can connect to it. The
+ // bound address will never be exposed.
+ if err := ep.Bind(tcpip.FullAddress{Addr: "dummy"}, nil); err != nil {
+ ep.Close()
+ return nil, err
+ }
+
+ // Create a connection from which the kernel can write messages.
+ connection, err := ep.(transport.BoundEndpoint).UnidirectionalConnect(t)
+ if err != nil {
+ ep.Close()
+ return nil, err
+ }
+
+ return &SocketVFS2{
+ socketOpsCommon: socketOpsCommon{
+ ports: t.Kernel().NetlinkPorts(),
+ protocol: protocol,
+ skType: skType,
+ ep: ep,
+ connection: connection,
+ sendBufferSize: defaultSendBufferSize,
+ },
+ }, nil
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (s *SocketVFS2) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return s.socketOpsCommon.Readiness(mask)
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (s *SocketVFS2) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ s.socketOpsCommon.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (s *SocketVFS2) EventUnregister(e *waiter.Entry) {
+ s.socketOpsCommon.EventUnregister(e)
+}
+
+// Ioctl implements vfs.FileDescriptionImpl.
+func (*SocketVFS2) Ioctl(context.Context, usermem.IO, arch.SyscallArguments) (uintptr, error) {
+ // TODO(b/68878065): no ioctls supported.
+ return 0, syserror.ENOTTY
+}
+
+// PRead implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
+ return 0, syserror.ESPIPE
+}
+
+// Read implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+ // All flags other than RWF_NOWAIT should be ignored.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
+ if opts.Flags != 0 {
+ return 0, syserror.EOPNOTSUPP
+ }
+
+ if dst.NumBytes() == 0 {
+ return 0, nil
+ }
+ return dst.CopyOutFrom(ctx, &unix.EndpointReader{
+ Endpoint: s.ep,
+ })
+}
+
+// PWrite implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
+ return 0, syserror.ESPIPE
+}
+
+// Write implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+ // All flags other than RWF_NOWAIT should be ignored.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
+ if opts.Flags != 0 {
+ return 0, syserror.EOPNOTSUPP
+ }
+
+ n, err := s.sendMsg(ctx, src, nil, 0, socket.ControlMessages{})
+ return int64(n), err.ToError()
+}
diff --git a/pkg/sentry/socket/netstack/BUILD b/pkg/sentry/socket/netstack/BUILD
index cbf46b1e9..ccf9fcf5c 100644
--- a/pkg/sentry/socket/netstack/BUILD
+++ b/pkg/sentry/socket/netstack/BUILD
@@ -7,7 +7,9 @@ go_library(
srcs = [
"device.go",
"netstack.go",
+ "netstack_vfs2.go",
"provider.go",
+ "provider_vfs2.go",
"save_restore.go",
"stack.go",
],
@@ -25,6 +27,8 @@ go_library(
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/fsimpl/kernfs",
+ "//pkg/sentry/fsimpl/sockfs",
"//pkg/sentry/inet",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
@@ -32,6 +36,7 @@ go_library(
"//pkg/sentry/socket",
"//pkg/sentry/socket/netfilter",
"//pkg/sentry/unimpl",
+ "//pkg/sentry/vfs",
"//pkg/sync",
"//pkg/syserr",
"//pkg/syserror",
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index d5879c10f..81053d8ef 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -252,6 +252,8 @@ type commonEndpoint interface {
GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error)
}
+// LINT.IfChange
+
// SocketOperations encapsulates all the state needed to represent a network stack
// endpoint in the kernel context.
//
@@ -263,6 +265,14 @@ type SocketOperations struct {
fsutil.FileNoFsync `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+
+ socketOpsCommon
+}
+
+// socketOpsCommon contains the socket operations common to VFS1 and VFS2.
+//
+// +stateify savable
+type socketOpsCommon struct {
socket.SendReceiveTimeout
*waiter.Queue
@@ -314,11 +324,13 @@ func New(t *kernel.Task, family int, skType linux.SockType, protocol int, queue
dirent := socket.NewDirent(t, netstackDevice)
defer dirent.DecRef()
return fs.NewFile(t, dirent, fs.FileFlags{Read: true, Write: true, NonSeekable: true}, &SocketOperations{
- Queue: queue,
- family: family,
- Endpoint: endpoint,
- skType: skType,
- protocol: protocol,
+ socketOpsCommon: socketOpsCommon{
+ Queue: queue,
+ family: family,
+ Endpoint: endpoint,
+ skType: skType,
+ protocol: protocol,
+ },
}), nil
}
@@ -417,7 +429,7 @@ func AddressAndFamily(addr []byte) (tcpip.FullAddress, uint16, *syserr.Error) {
}
}
-func (s *SocketOperations) isPacketBased() bool {
+func (s *socketOpsCommon) isPacketBased() bool {
return s.skType == linux.SOCK_DGRAM || s.skType == linux.SOCK_SEQPACKET || s.skType == linux.SOCK_RDM || s.skType == linux.SOCK_RAW
}
@@ -425,7 +437,7 @@ func (s *SocketOperations) isPacketBased() bool {
// empty. It assumes that the socket is locked.
//
// Precondition: s.readMu must be held.
-func (s *SocketOperations) fetchReadView() *syserr.Error {
+func (s *socketOpsCommon) fetchReadView() *syserr.Error {
if len(s.readView) > 0 {
return nil
}
@@ -446,7 +458,7 @@ func (s *SocketOperations) fetchReadView() *syserr.Error {
}
// Release implements fs.FileOperations.Release.
-func (s *SocketOperations) Release() {
+func (s *socketOpsCommon) Release() {
s.Endpoint.Close()
}
@@ -633,7 +645,7 @@ func (s *SocketOperations) ReadFrom(ctx context.Context, _ *fs.File, r io.Reader
}
// Readiness returns a mask of ready events for socket s.
-func (s *SocketOperations) Readiness(mask waiter.EventMask) waiter.EventMask {
+func (s *socketOpsCommon) Readiness(mask waiter.EventMask) waiter.EventMask {
r := s.Endpoint.Readiness(mask)
// Check our cached value iff the caller asked for readability and the
@@ -647,7 +659,7 @@ func (s *SocketOperations) Readiness(mask waiter.EventMask) waiter.EventMask {
return r
}
-func (s *SocketOperations) checkFamily(family uint16, exact bool) *syserr.Error {
+func (s *socketOpsCommon) checkFamily(family uint16, exact bool) *syserr.Error {
if family == uint16(s.family) {
return nil
}
@@ -670,7 +682,7 @@ func (s *SocketOperations) checkFamily(family uint16, exact bool) *syserr.Error
// represented by the empty string.
//
// TODO(gvisor.dev/issue/1556): remove this function.
-func (s *SocketOperations) mapFamily(addr tcpip.FullAddress, family uint16) tcpip.FullAddress {
+func (s *socketOpsCommon) mapFamily(addr tcpip.FullAddress, family uint16) tcpip.FullAddress {
if len(addr.Addr) == 0 && s.family == linux.AF_INET6 && family == linux.AF_INET {
addr.Addr = "\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xff\xff\x00\x00\x00\x00"
}
@@ -679,7 +691,7 @@ func (s *SocketOperations) mapFamily(addr tcpip.FullAddress, family uint16) tcpi
// Connect implements the linux syscall connect(2) for sockets backed by
// tpcip.Endpoint.
-func (s *SocketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
+func (s *socketOpsCommon) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
addr, family, err := AddressAndFamily(sockaddr)
if err != nil {
return err
@@ -725,7 +737,7 @@ func (s *SocketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking boo
// Bind implements the linux syscall bind(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
+func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
if len(sockaddr) < 2 {
return syserr.ErrInvalidArgument
}
@@ -771,13 +783,13 @@ func (s *SocketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
// Listen implements the linux syscall listen(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error {
+func (s *socketOpsCommon) Listen(t *kernel.Task, backlog int) *syserr.Error {
return syserr.TranslateNetstackError(s.Endpoint.Listen(backlog))
}
// blockingAccept implements a blocking version of accept(2), that is, if no
// connections are ready to be accept, it will block until one becomes ready.
-func (s *SocketOperations) blockingAccept(t *kernel.Task) (tcpip.Endpoint, *waiter.Queue, *syserr.Error) {
+func (s *socketOpsCommon) blockingAccept(t *kernel.Task) (tcpip.Endpoint, *waiter.Queue, *syserr.Error) {
// Register for notifications.
e, ch := waiter.NewChannelEntry(nil)
s.EventRegister(&e, waiter.EventIn)
@@ -863,7 +875,7 @@ func ConvertShutdown(how int) (tcpip.ShutdownFlags, *syserr.Error) {
// Shutdown implements the linux syscall shutdown(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error {
+func (s *socketOpsCommon) Shutdown(t *kernel.Task, how int) *syserr.Error {
f, err := ConvertShutdown(how)
if err != nil {
return err
@@ -2258,7 +2270,7 @@ func ConvertAddress(family int, addr tcpip.FullAddress) (linux.SockAddr, uint32)
// GetSockName implements the linux syscall getsockname(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
addr, err := s.Endpoint.GetLocalAddress()
if err != nil {
return nil, 0, syserr.TranslateNetstackError(err)
@@ -2270,7 +2282,7 @@ func (s *SocketOperations) GetSockName(t *kernel.Task) (linux.SockAddr, uint32,
// GetPeerName implements the linux syscall getpeername(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
addr, err := s.Endpoint.GetRemoteAddress()
if err != nil {
return nil, 0, syserr.TranslateNetstackError(err)
@@ -2285,7 +2297,7 @@ func (s *SocketOperations) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32,
// caller.
//
// Precondition: s.readMu must be locked.
-func (s *SocketOperations) coalescingRead(ctx context.Context, dst usermem.IOSequence, discard bool) (int, *syserr.Error) {
+func (s *socketOpsCommon) coalescingRead(ctx context.Context, dst usermem.IOSequence, discard bool) (int, *syserr.Error) {
var err *syserr.Error
var copied int
@@ -2337,7 +2349,7 @@ func (s *SocketOperations) coalescingRead(ctx context.Context, dst usermem.IOSeq
return 0, err
}
-func (s *SocketOperations) fillCmsgInq(cmsg *socket.ControlMessages) {
+func (s *socketOpsCommon) fillCmsgInq(cmsg *socket.ControlMessages) {
if !s.sockOptInq {
return
}
@@ -2352,7 +2364,7 @@ func (s *SocketOperations) fillCmsgInq(cmsg *socket.ControlMessages) {
// nonBlockingRead issues a non-blocking read.
//
// TODO(b/78348848): Support timestamps for stream sockets.
-func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, int, linux.SockAddr, uint32, socket.ControlMessages, *syserr.Error) {
+func (s *socketOpsCommon) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, int, linux.SockAddr, uint32, socket.ControlMessages, *syserr.Error) {
isPacket := s.isPacketBased()
// Fast path for regular reads from stream (e.g., TCP) endpoints. Note
@@ -2461,7 +2473,7 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe
return n, flags, addr, addrLen, cmsg, syserr.FromError(err)
}
-func (s *SocketOperations) controlMessages() socket.ControlMessages {
+func (s *socketOpsCommon) controlMessages() socket.ControlMessages {
return socket.ControlMessages{
IP: tcpip.ControlMessages{
HasTimestamp: s.readCM.HasTimestamp && s.sockOptTimestamp,
@@ -2480,7 +2492,7 @@ func (s *SocketOperations) controlMessages() socket.ControlMessages {
// successfully writing packet data out to userspace.
//
// Precondition: s.readMu must be locked.
-func (s *SocketOperations) updateTimestamp() {
+func (s *socketOpsCommon) updateTimestamp() {
// Save the SIOCGSTAMP timestamp only if SO_TIMESTAMP is disabled.
if !s.sockOptTimestamp {
s.timestampValid = true
@@ -2490,7 +2502,7 @@ func (s *SocketOperations) updateTimestamp() {
// RecvMsg implements the linux syscall recvmsg(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, msgFlags int, senderAddr linux.SockAddr, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) {
+func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, msgFlags int, senderAddr linux.SockAddr, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) {
trunc := flags&linux.MSG_TRUNC != 0
peek := flags&linux.MSG_PEEK != 0
dontWait := flags&linux.MSG_DONTWAIT != 0
@@ -2558,7 +2570,7 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags
// SendMsg implements the linux syscall sendmsg(2) for sockets backed by
// tcpip.Endpoint.
-func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) {
+func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) {
// Reject Unix control messages.
if !controlMessages.Unix.Empty() {
return 0, syserr.ErrInvalidArgument
@@ -2634,6 +2646,10 @@ func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []
// Ioctl implements fs.FileOperations.Ioctl.
func (s *SocketOperations) Ioctl(ctx context.Context, _ *fs.File, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ return s.socketOpsCommon.ioctl(ctx, io, args)
+}
+
+func (s *socketOpsCommon) ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
// SIOCGSTAMP is implemented by netstack rather than all commonEndpoint
// sockets.
// TODO(b/78348848): Add a commonEndpoint method to support SIOCGSTAMP.
@@ -2973,7 +2989,7 @@ func nicStateFlagsToLinux(f stack.NICStateFlags) uint32 {
// State implements socket.Socket.State. State translates the internal state
// returned by netstack to values defined by Linux.
-func (s *SocketOperations) State() uint32 {
+func (s *socketOpsCommon) State() uint32 {
if s.family != linux.AF_INET && s.family != linux.AF_INET6 {
// States not implemented for this socket's family.
return 0
@@ -3033,6 +3049,8 @@ func (s *SocketOperations) State() uint32 {
}
// Type implements socket.Socket.Type.
-func (s *SocketOperations) Type() (family int, skType linux.SockType, protocol int) {
+func (s *socketOpsCommon) Type() (family int, skType linux.SockType, protocol int) {
return s.family, s.skType, s.protocol
}
+
+// LINT.ThenChange(./netstack_vfs2.go)
diff --git a/pkg/sentry/socket/netstack/netstack_vfs2.go b/pkg/sentry/socket/netstack/netstack_vfs2.go
new file mode 100644
index 000000000..eec71035d
--- /dev/null
+++ b/pkg/sentry/socket/netstack/netstack_vfs2.go
@@ -0,0 +1,327 @@
+// Copyright 2018 The gVisor Authors.
+//
+// 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.
+
+package netstack
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/sockfs"
+ "gvisor.dev/gvisor/pkg/sentry/inet"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/socket/netfilter"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserr"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/tcpip"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// SocketVFS2 encapsulates all the state needed to represent a network stack
+// endpoint in the kernel context.
+type SocketVFS2 struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ socketOpsCommon
+}
+
+// NewVFS2 creates a new endpoint socket.
+func NewVFS2(t *kernel.Task, family int, skType linux.SockType, protocol int, queue *waiter.Queue, endpoint tcpip.Endpoint) (*vfs.FileDescription, *syserr.Error) {
+ if skType == linux.SOCK_STREAM {
+ if err := endpoint.SetSockOptBool(tcpip.DelayOption, true); err != nil {
+ return nil, syserr.TranslateNetstackError(err)
+ }
+ }
+
+ mnt := t.Kernel().SocketMount()
+ fs := mnt.Filesystem().Impl().(*kernfs.Filesystem)
+ d := sockfs.NewDentry(t.Credentials(), fs.NextIno())
+
+ s := &SocketVFS2{
+ socketOpsCommon: socketOpsCommon{
+ Queue: queue,
+ family: family,
+ Endpoint: endpoint,
+ skType: skType,
+ protocol: protocol,
+ },
+ }
+ vfsfd := &s.vfsfd
+ if err := vfsfd.Init(s, linux.O_RDWR, mnt, d, &vfs.FileDescriptionOptions{
+ DenyPRead: true,
+ DenyPWrite: true,
+ UseDentryMetadata: true,
+ }); err != nil {
+ return nil, syserr.FromError(err)
+ }
+ return vfsfd, nil
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (s *SocketVFS2) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return s.socketOpsCommon.Readiness(mask)
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (s *SocketVFS2) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ s.socketOpsCommon.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (s *SocketVFS2) EventUnregister(e *waiter.Entry) {
+ s.socketOpsCommon.EventUnregister(e)
+}
+
+// PRead implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
+ return 0, syserror.ESPIPE
+}
+
+// Read implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+ // All flags other than RWF_NOWAIT should be ignored.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
+ if opts.Flags != 0 {
+ return 0, syserror.EOPNOTSUPP
+ }
+
+ if dst.NumBytes() == 0 {
+ return 0, nil
+ }
+ n, _, _, _, _, err := s.nonBlockingRead(ctx, dst, false, false, false)
+ if err == syserr.ErrWouldBlock {
+ return int64(n), syserror.ErrWouldBlock
+ }
+ if err != nil {
+ return 0, err.ToError()
+ }
+ return int64(n), nil
+}
+
+// PWrite implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
+ return 0, syserror.ESPIPE
+}
+
+// Write implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+ // All flags other than RWF_NOWAIT should be ignored.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
+ if opts.Flags != 0 {
+ return 0, syserror.EOPNOTSUPP
+ }
+
+ f := &ioSequencePayload{ctx: ctx, src: src}
+ n, resCh, err := s.Endpoint.Write(f, tcpip.WriteOptions{})
+ if err == tcpip.ErrWouldBlock {
+ return 0, syserror.ErrWouldBlock
+ }
+
+ if resCh != nil {
+ t := kernel.TaskFromContext(ctx)
+ if err := t.Block(resCh); err != nil {
+ return 0, syserr.FromError(err).ToError()
+ }
+
+ n, _, err = s.Endpoint.Write(f, tcpip.WriteOptions{})
+ }
+
+ if err != nil {
+ return 0, syserr.TranslateNetstackError(err).ToError()
+ }
+
+ if int64(n) < src.NumBytes() {
+ return int64(n), syserror.ErrWouldBlock
+ }
+
+ return int64(n), nil
+}
+
+// Accept implements the linux syscall accept(2) for sockets backed by
+// tcpip.Endpoint.
+func (s *SocketVFS2) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (int32, linux.SockAddr, uint32, *syserr.Error) {
+ // Issue the accept request to get the new endpoint.
+ ep, wq, terr := s.Endpoint.Accept()
+ if terr != nil {
+ if terr != tcpip.ErrWouldBlock || !blocking {
+ return 0, nil, 0, syserr.TranslateNetstackError(terr)
+ }
+
+ var err *syserr.Error
+ ep, wq, err = s.blockingAccept(t)
+ if err != nil {
+ return 0, nil, 0, err
+ }
+ }
+
+ ns, err := NewVFS2(t, s.family, s.skType, s.protocol, wq, ep)
+ if err != nil {
+ return 0, nil, 0, err
+ }
+ defer ns.DecRef()
+
+ if err := ns.SetStatusFlags(t, t.Credentials(), uint32(flags&linux.SOCK_NONBLOCK)); err != nil {
+ return 0, nil, 0, syserr.FromError(err)
+ }
+
+ var addr linux.SockAddr
+ var addrLen uint32
+ if peerRequested {
+ // Get address of the peer and write it to peer slice.
+ var err *syserr.Error
+ addr, addrLen, err = ns.Impl().(*SocketVFS2).GetPeerName(t)
+ if err != nil {
+ return 0, nil, 0, err
+ }
+ }
+
+ fd, e := t.NewFDFromVFS2(0, ns, kernel.FDFlags{
+ CloseOnExec: flags&linux.SOCK_CLOEXEC != 0,
+ })
+
+ t.Kernel().RecordSocketVFS2(ns)
+
+ return fd, addr, addrLen, syserr.FromError(e)
+}
+
+// Ioctl implements vfs.FileDescriptionImpl.
+func (s *SocketVFS2) Ioctl(ctx context.Context, uio usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ return s.socketOpsCommon.ioctl(ctx, uio, args)
+}
+
+// GetSockOpt implements the linux syscall getsockopt(2) for sockets backed by
+// tcpip.Endpoint.
+func (s *SocketVFS2) GetSockOpt(t *kernel.Task, level, name int, outPtr usermem.Addr, outLen int) (interface{}, *syserr.Error) {
+ // TODO(b/78348848): Unlike other socket options, SO_TIMESTAMP is
+ // implemented specifically for netstack.SocketVFS2 rather than
+ // commonEndpoint. commonEndpoint should be extended to support socket
+ // options where the implementation is not shared, as unix sockets need
+ // their own support for SO_TIMESTAMP.
+ if level == linux.SOL_SOCKET && name == linux.SO_TIMESTAMP {
+ if outLen < sizeOfInt32 {
+ return nil, syserr.ErrInvalidArgument
+ }
+ val := int32(0)
+ s.readMu.Lock()
+ defer s.readMu.Unlock()
+ if s.sockOptTimestamp {
+ val = 1
+ }
+ return val, nil
+ }
+ if level == linux.SOL_TCP && name == linux.TCP_INQ {
+ if outLen < sizeOfInt32 {
+ return nil, syserr.ErrInvalidArgument
+ }
+ val := int32(0)
+ s.readMu.Lock()
+ defer s.readMu.Unlock()
+ if s.sockOptInq {
+ val = 1
+ }
+ return val, nil
+ }
+
+ if s.skType == linux.SOCK_RAW && level == linux.IPPROTO_IP {
+ switch name {
+ case linux.IPT_SO_GET_INFO:
+ if outLen < linux.SizeOfIPTGetinfo {
+ return nil, syserr.ErrInvalidArgument
+ }
+
+ stack := inet.StackFromContext(t)
+ if stack == nil {
+ return nil, syserr.ErrNoDevice
+ }
+ info, err := netfilter.GetInfo(t, stack.(*Stack).Stack, outPtr)
+ if err != nil {
+ return nil, err
+ }
+ return info, nil
+
+ case linux.IPT_SO_GET_ENTRIES:
+ if outLen < linux.SizeOfIPTGetEntries {
+ return nil, syserr.ErrInvalidArgument
+ }
+
+ stack := inet.StackFromContext(t)
+ if stack == nil {
+ return nil, syserr.ErrNoDevice
+ }
+ entries, err := netfilter.GetEntries(t, stack.(*Stack).Stack, outPtr, outLen)
+ if err != nil {
+ return nil, err
+ }
+ return entries, nil
+
+ }
+ }
+
+ return GetSockOpt(t, s, s.Endpoint, s.family, s.skType, level, name, outLen)
+}
+
+// SetSockOpt implements the linux syscall setsockopt(2) for sockets backed by
+// tcpip.Endpoint.
+func (s *SocketVFS2) SetSockOpt(t *kernel.Task, level int, name int, optVal []byte) *syserr.Error {
+ // TODO(b/78348848): Unlike other socket options, SO_TIMESTAMP is
+ // implemented specifically for netstack.SocketVFS2 rather than
+ // commonEndpoint. commonEndpoint should be extended to support socket
+ // options where the implementation is not shared, as unix sockets need
+ // their own support for SO_TIMESTAMP.
+ if level == linux.SOL_SOCKET && name == linux.SO_TIMESTAMP {
+ if len(optVal) < sizeOfInt32 {
+ return syserr.ErrInvalidArgument
+ }
+ s.readMu.Lock()
+ defer s.readMu.Unlock()
+ s.sockOptTimestamp = usermem.ByteOrder.Uint32(optVal) != 0
+ return nil
+ }
+ if level == linux.SOL_TCP && name == linux.TCP_INQ {
+ if len(optVal) < sizeOfInt32 {
+ return syserr.ErrInvalidArgument
+ }
+ s.readMu.Lock()
+ defer s.readMu.Unlock()
+ s.sockOptInq = usermem.ByteOrder.Uint32(optVal) != 0
+ return nil
+ }
+
+ if s.skType == linux.SOCK_RAW && level == linux.IPPROTO_IP {
+ switch name {
+ case linux.IPT_SO_SET_REPLACE:
+ if len(optVal) < linux.SizeOfIPTReplace {
+ return syserr.ErrInvalidArgument
+ }
+
+ stack := inet.StackFromContext(t)
+ if stack == nil {
+ return syserr.ErrNoDevice
+ }
+ // Stack must be a netstack stack.
+ return netfilter.SetEntries(stack.(*Stack).Stack, optVal)
+
+ case linux.IPT_SO_SET_ADD_COUNTERS:
+ // TODO(gvisor.dev/issue/170): Counter support.
+ return nil
+ }
+ }
+
+ return SetSockOpt(t, s, s.Endpoint, level, name, optVal)
+}
diff --git a/pkg/sentry/socket/netstack/provider.go b/pkg/sentry/socket/netstack/provider.go
index c3f04b613..ead3b2b79 100644
--- a/pkg/sentry/socket/netstack/provider.go
+++ b/pkg/sentry/socket/netstack/provider.go
@@ -33,6 +33,8 @@ import (
"gvisor.dev/gvisor/pkg/waiter"
)
+// LINT.IfChange
+
// provider is an inet socket provider.
type provider struct {
family int
@@ -167,6 +169,8 @@ func packetSocket(t *kernel.Task, epStack *Stack, stype linux.SockType, protocol
return New(t, linux.AF_PACKET, stype, protocol, wq, ep)
}
+// LINT.ThenChange(./provider_vfs2.go)
+
// Pair just returns nil sockets (not supported).
func (*provider) Pair(*kernel.Task, linux.SockType, int) (*fs.File, *fs.File, *syserr.Error) {
return nil, nil, nil
diff --git a/pkg/sentry/socket/netstack/provider_vfs2.go b/pkg/sentry/socket/netstack/provider_vfs2.go
new file mode 100644
index 000000000..2a01143f6
--- /dev/null
+++ b/pkg/sentry/socket/netstack/provider_vfs2.go
@@ -0,0 +1,141 @@
+// Copyright 2020 The gVisor Authors.
+//
+// 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.
+
+package netstack
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/socket"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserr"
+ "gvisor.dev/gvisor/pkg/tcpip"
+ "gvisor.dev/gvisor/pkg/tcpip/network/ipv4"
+ "gvisor.dev/gvisor/pkg/tcpip/network/ipv6"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// providerVFS2 is an inet socket provider.
+type providerVFS2 struct {
+ family int
+ netProto tcpip.NetworkProtocolNumber
+}
+
+// Socket creates a new socket object for the AF_INET, AF_INET6, or AF_PACKET
+// family.
+func (p *providerVFS2) Socket(t *kernel.Task, stype linux.SockType, protocol int) (*vfs.FileDescription, *syserr.Error) {
+ // Fail right away if we don't have a stack.
+ stack := t.NetworkContext()
+ if stack == nil {
+ // Don't propagate an error here. Instead, allow the socket
+ // code to continue searching for another provider.
+ return nil, nil
+ }
+ eps, ok := stack.(*Stack)
+ if !ok {
+ return nil, nil
+ }
+
+ // Packet sockets are handled separately, since they are neither INET
+ // nor INET6 specific.
+ if p.family == linux.AF_PACKET {
+ return packetSocketVFS2(t, eps, stype, protocol)
+ }
+
+ // Figure out the transport protocol.
+ transProto, associated, err := getTransportProtocol(t, stype, protocol)
+ if err != nil {
+ return nil, err
+ }
+
+ // Create the endpoint.
+ var ep tcpip.Endpoint
+ var e *tcpip.Error
+ wq := &waiter.Queue{}
+ if stype == linux.SOCK_RAW {
+ ep, e = eps.Stack.NewRawEndpoint(transProto, p.netProto, wq, associated)
+ } else {
+ ep, e = eps.Stack.NewEndpoint(transProto, p.netProto, wq)
+
+ // Assign task to PacketOwner interface to get the UID and GID for
+ // iptables owner matching.
+ if e == nil {
+ ep.SetOwner(t)
+ }
+ }
+ if e != nil {
+ return nil, syserr.TranslateNetstackError(e)
+ }
+
+ return NewVFS2(t, p.family, stype, int(transProto), wq, ep)
+}
+
+func packetSocketVFS2(t *kernel.Task, epStack *Stack, stype linux.SockType, protocol int) (*vfs.FileDescription, *syserr.Error) {
+ // Packet sockets require CAP_NET_RAW.
+ creds := auth.CredentialsFromContext(t)
+ if !creds.HasCapability(linux.CAP_NET_RAW) {
+ return nil, syserr.ErrNotPermitted
+ }
+
+ // "cooked" packets don't contain link layer information.
+ var cooked bool
+ switch stype {
+ case linux.SOCK_DGRAM:
+ cooked = true
+ case linux.SOCK_RAW:
+ cooked = false
+ default:
+ return nil, syserr.ErrProtocolNotSupported
+ }
+
+ // protocol is passed in network byte order, but netstack wants it in
+ // host order.
+ netProto := tcpip.NetworkProtocolNumber(ntohs(uint16(protocol)))
+
+ wq := &waiter.Queue{}
+ ep, err := epStack.Stack.NewPacketEndpoint(cooked, netProto, wq)
+ if err != nil {
+ return nil, syserr.TranslateNetstackError(err)
+ }
+
+ return NewVFS2(t, linux.AF_PACKET, stype, protocol, wq, ep)
+}
+
+// Pair just returns nil sockets (not supported).
+func (*providerVFS2) Pair(*kernel.Task, linux.SockType, int) (*vfs.FileDescription, *vfs.FileDescription, *syserr.Error) {
+ return nil, nil, nil
+}
+
+// init registers socket providers for AF_INET, AF_INET6, and AF_PACKET.
+func init() {
+ // Providers backed by netstack.
+ p := []providerVFS2{
+ {
+ family: linux.AF_INET,
+ netProto: ipv4.ProtocolNumber,
+ },
+ {
+ family: linux.AF_INET6,
+ netProto: ipv6.ProtocolNumber,
+ },
+ {
+ family: linux.AF_PACKET,
+ },
+ }
+
+ for i := range p {
+ socket.RegisterProviderVFS2(p[i].family, &p[i])
+ }
+}
diff --git a/pkg/sentry/socket/unix/unix_vfs2.go b/pkg/sentry/socket/unix/unix_vfs2.go
index 23db93f33..5edc3cdf4 100644
--- a/pkg/sentry/socket/unix/unix_vfs2.go
+++ b/pkg/sentry/socket/unix/unix_vfs2.go
@@ -229,7 +229,7 @@ func (s *SocketVFS2) PRead(ctx context.Context, dst usermem.IOSequence, offset i
// Read implements vfs.FileDescriptionImpl.
func (s *SocketVFS2) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
// All flags other than RWF_NOWAIT should be ignored.
- // TODO(gvisor.dev/issue/1476): Support RWF_NOWAIT.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
if opts.Flags != 0 {
return 0, syserror.EOPNOTSUPP
}
@@ -254,7 +254,7 @@ func (s *SocketVFS2) PWrite(ctx context.Context, src usermem.IOSequence, offset
// Write implements vfs.FileDescriptionImpl.
func (s *SocketVFS2) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
// All flags other than RWF_NOWAIT should be ignored.
- // TODO(gvisor.dev/issue/1476): Support RWF_NOWAIT.
+ // TODO(gvisor.dev/issue/2601): Support RWF_NOWAIT.
if opts.Flags != 0 {
return 0, syserror.EOPNOTSUPP
}