summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fsimpl')
-rw-r--r--pkg/sentry/fsimpl/devpts/devpts.go2
-rw-r--r--pkg/sentry/fsimpl/ext/filesystem.go7
-rw-r--r--pkg/sentry/fsimpl/gofer/BUILD8
-rw-r--r--pkg/sentry/fsimpl/gofer/directory.go39
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go130
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go92
-rw-r--r--pkg/sentry/fsimpl/gofer/socket.go146
-rw-r--r--pkg/sentry/fsimpl/host/BUILD1
-rw-r--r--pkg/sentry/fsimpl/host/host.go125
-rw-r--r--pkg/sentry/fsimpl/host/socket.go53
-rw-r--r--pkg/sentry/fsimpl/host/tty.go26
-rw-r--r--pkg/sentry/fsimpl/kernfs/fd_impl_util.go30
-rw-r--r--pkg/sentry/fsimpl/kernfs/filesystem.go7
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs.go12
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs_test.go2
-rw-r--r--pkg/sentry/fsimpl/pipefs/pipefs.go28
-rw-r--r--pkg/sentry/fsimpl/proc/task_net.go2
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_files.go10
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_sys.go4
-rw-r--r--pkg/sentry/fsimpl/sockfs/sockfs.go13
-rw-r--r--pkg/sentry/fsimpl/sys/sys.go2
-rw-r--r--pkg/sentry/fsimpl/tmpfs/filesystem.go5
22 files changed, 548 insertions, 196 deletions
diff --git a/pkg/sentry/fsimpl/devpts/devpts.go b/pkg/sentry/fsimpl/devpts/devpts.go
index 181d765d3..94db8fe5c 100644
--- a/pkg/sentry/fsimpl/devpts/devpts.go
+++ b/pkg/sentry/fsimpl/devpts/devpts.go
@@ -59,7 +59,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
// master inode. It returns the filesystem and root Dentry.
func (fstype FilesystemType) newFilesystem(vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials) (*kernfs.Filesystem, *kernfs.Dentry) {
fs := &kernfs.Filesystem{}
- fs.Init(vfsObj, fstype)
+ fs.VFSFilesystem().Init(vfsObj, fstype, fs)
// Construct the root directory. This is always inode id 1.
root := &rootInode{
diff --git a/pkg/sentry/fsimpl/ext/filesystem.go b/pkg/sentry/fsimpl/ext/filesystem.go
index 2c22a04af..77b644275 100644
--- a/pkg/sentry/fsimpl/ext/filesystem.go
+++ b/pkg/sentry/fsimpl/ext/filesystem.go
@@ -485,11 +485,14 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
-func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
- _, _, err := fs.walk(rp, false)
+func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
+ _, inode, err := fs.walk(rp, false)
if err != nil {
return nil, err
}
+ if err := inode.checkPermissions(rp.Credentials(), vfs.MayWrite); err != nil {
+ return nil, err
+ }
// TODO(b/134676337): Support sockets.
return nil, syserror.ECONNREFUSED
diff --git a/pkg/sentry/fsimpl/gofer/BUILD b/pkg/sentry/fsimpl/gofer/BUILD
index b9c4beee4..5ce82b793 100644
--- a/pkg/sentry/fsimpl/gofer/BUILD
+++ b/pkg/sentry/fsimpl/gofer/BUILD
@@ -38,6 +38,7 @@ go_library(
"p9file.go",
"pagemath.go",
"regular_file.go",
+ "socket.go",
"special_file.go",
"symlink.go",
"time.go",
@@ -52,18 +53,25 @@ go_library(
"//pkg/p9",
"//pkg/safemem",
"//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/fsimpl/host",
"//pkg/sentry/hostfd",
+ "//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/time",
"//pkg/sentry/memmap",
"//pkg/sentry/pgalloc",
"//pkg/sentry/platform",
+ "//pkg/sentry/socket/control",
+ "//pkg/sentry/socket/unix",
"//pkg/sentry/socket/unix/transport",
"//pkg/sentry/usage",
"//pkg/sentry/vfs",
+ "//pkg/syserr",
"//pkg/syserror",
"//pkg/unet",
"//pkg/usermem",
+ "//pkg/waiter",
],
)
diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go
index 55f9ed911..b98218753 100644
--- a/pkg/sentry/fsimpl/gofer/directory.go
+++ b/pkg/sentry/fsimpl/gofer/directory.go
@@ -15,6 +15,7 @@
package gofer
import (
+ "fmt"
"sync"
"sync/atomic"
@@ -22,6 +23,8 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
+ "gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
@@ -59,28 +62,52 @@ func (d *dentry) cacheNegativeLookupLocked(name string) {
d.children[name] = nil
}
-// createSyntheticDirectory creates a synthetic directory with the given name
+type createSyntheticOpts struct {
+ name string
+ mode linux.FileMode
+ kuid auth.KUID
+ kgid auth.KGID
+
+ // The endpoint for a synthetic socket. endpoint should be nil if the file
+ // being created is not a socket.
+ endpoint transport.BoundEndpoint
+
+ // pipe should be nil if the file being created is not a pipe.
+ pipe *pipe.VFSPipe
+}
+
+// createSyntheticChildLocked creates a synthetic file with the given name
// in d.
//
// Preconditions: d.dirMu must be locked. d.isDir(). d does not already contain
// a child with the given name.
-func (d *dentry) createSyntheticDirectoryLocked(name string, mode linux.FileMode, kuid auth.KUID, kgid auth.KGID) {
+func (d *dentry) createSyntheticChildLocked(opts *createSyntheticOpts) {
d2 := &dentry{
refs: 1, // held by d
fs: d.fs,
- mode: uint32(mode) | linux.S_IFDIR,
- uid: uint32(kuid),
- gid: uint32(kgid),
+ mode: uint32(opts.mode),
+ uid: uint32(opts.kuid),
+ gid: uint32(opts.kgid),
blockSize: usermem.PageSize, // arbitrary
handle: handle{
fd: -1,
},
nlink: uint32(2),
}
+ switch opts.mode.FileType() {
+ case linux.S_IFDIR:
+ // Nothing else needs to be done.
+ case linux.S_IFSOCK:
+ d2.endpoint = opts.endpoint
+ case linux.S_IFIFO:
+ d2.pipe = opts.pipe
+ default:
+ panic(fmt.Sprintf("failed to create synthetic file of unrecognized type: %v", opts.mode.FileType()))
+ }
d2.pf.dentry = d2
d2.vfsd.Init(d2)
- d.cacheNewChildLocked(d2, name)
+ d.cacheNewChildLocked(d2, opts.name)
d.syntheticChildren++
}
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 98ccb42fd..4a32821bd 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -22,9 +22,11 @@ import (
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
)
// Sync implements vfs.FilesystemImpl.Sync.
@@ -652,7 +654,12 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
return err
}
ctx.Infof("Failed to create remote directory %q: %v; falling back to synthetic directory", name, err)
- parent.createSyntheticDirectoryLocked(name, opts.Mode, creds.EffectiveKUID, creds.EffectiveKGID)
+ parent.createSyntheticChildLocked(&createSyntheticOpts{
+ name: name,
+ mode: linux.S_IFDIR | opts.Mode,
+ kuid: creds.EffectiveKUID,
+ kgid: creds.EffectiveKGID,
+ })
}
if fs.opts.interop != InteropModeShared {
parent.incLinks()
@@ -663,7 +670,12 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
// Can't create non-synthetic files in synthetic directories.
return syserror.EPERM
}
- parent.createSyntheticDirectoryLocked(name, opts.Mode, creds.EffectiveKUID, creds.EffectiveKGID)
+ parent.createSyntheticChildLocked(&createSyntheticOpts{
+ name: name,
+ mode: linux.S_IFDIR | opts.Mode,
+ kuid: creds.EffectiveKUID,
+ kgid: creds.EffectiveKGID,
+ })
parent.incLinks()
return nil
})
@@ -674,6 +686,30 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string) error {
creds := rp.Credentials()
_, err := parent.file.mknod(ctx, name, (p9.FileMode)(opts.Mode), opts.DevMajor, opts.DevMinor, (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
+ // If the gofer does not allow creating a socket or pipe, create a
+ // synthetic one, i.e. one that is kept entirely in memory.
+ if err == syserror.EPERM {
+ switch opts.Mode.FileType() {
+ case linux.S_IFSOCK:
+ parent.createSyntheticChildLocked(&createSyntheticOpts{
+ name: name,
+ mode: opts.Mode,
+ kuid: creds.EffectiveKUID,
+ kgid: creds.EffectiveKGID,
+ endpoint: opts.Endpoint,
+ })
+ return nil
+ case linux.S_IFIFO:
+ parent.createSyntheticChildLocked(&createSyntheticOpts{
+ name: name,
+ mode: opts.Mode,
+ kuid: creds.EffectiveKUID,
+ kgid: creds.EffectiveKGID,
+ pipe: pipe.NewVFSPipe(true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize),
+ })
+ return nil
+ }
+ }
return err
}, nil)
}
@@ -756,20 +792,21 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return nil, err
}
mnt := rp.Mount()
- filetype := d.fileType()
- switch {
- case filetype == linux.S_IFREG && !d.fs.opts.regularFilesUseSpecialFileFD:
- if err := d.ensureSharedHandle(ctx, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, opts.Flags&linux.O_TRUNC != 0); err != nil {
- return nil, err
- }
- fd := &regularFileFD{}
- if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{
- AllowDirectIO: true,
- }); err != nil {
- return nil, err
+ switch d.fileType() {
+ case linux.S_IFREG:
+ if !d.fs.opts.regularFilesUseSpecialFileFD {
+ if err := d.ensureSharedHandle(ctx, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, opts.Flags&linux.O_TRUNC != 0); err != nil {
+ return nil, err
+ }
+ fd := &regularFileFD{}
+ if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{
+ AllowDirectIO: true,
+ }); err != nil {
+ return nil, err
+ }
+ return &fd.vfsfd, nil
}
- return &fd.vfsfd, nil
- case filetype == linux.S_IFDIR:
+ case linux.S_IFDIR:
// Can't open directories with O_CREAT.
if opts.Flags&linux.O_CREAT != 0 {
return nil, syserror.EISDIR
@@ -791,26 +828,40 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return nil, err
}
return &fd.vfsfd, nil
- case filetype == linux.S_IFLNK:
+ case linux.S_IFLNK:
// Can't open symlinks without O_PATH (which is unimplemented).
return nil, syserror.ELOOP
- default:
- if opts.Flags&linux.O_DIRECT != 0 {
- return nil, syserror.EINVAL
- }
- h, err := openHandle(ctx, d.file, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, opts.Flags&linux.O_TRUNC != 0)
- if err != nil {
- return nil, err
- }
- fd := &specialFileFD{
- handle: h,
+ case linux.S_IFSOCK:
+ if d.isSynthetic() {
+ return nil, syserror.ENXIO
}
- if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{}); err != nil {
- h.close(ctx)
- return nil, err
+ case linux.S_IFIFO:
+ if d.isSynthetic() {
+ return d.pipe.Open(ctx, mnt, &d.vfsd, opts.Flags)
}
- return &fd.vfsfd, nil
}
+ return d.openSpecialFileLocked(ctx, mnt, opts)
+}
+
+func (d *dentry) openSpecialFileLocked(ctx context.Context, mnt *vfs.Mount, opts *vfs.OpenOptions) (*vfs.FileDescription, error) {
+ ats := vfs.AccessTypesForOpenFlags(opts)
+ // Treat as a special file. This is done for non-synthetic pipes as well as
+ // regular files when d.fs.opts.regularFilesUseSpecialFileFD is true.
+ if opts.Flags&linux.O_DIRECT != 0 {
+ return nil, syserror.EINVAL
+ }
+ h, err := openHandle(ctx, d.file, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, opts.Flags&linux.O_TRUNC != 0)
+ if err != nil {
+ return nil, err
+ }
+ fd := &specialFileFD{
+ handle: h,
+ }
+ if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{}); err != nil {
+ h.close(ctx)
+ return nil, err
+ }
+ return &fd.vfsfd, nil
}
// Preconditions: d.fs.renameMu must be locked. d.dirMu must be locked.
@@ -1196,15 +1247,28 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
-func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
+func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
var ds *[]*dentry
fs.renameMu.RLock()
defer fs.renameMuRUnlockAndCheckCaching(&ds)
- _, err := fs.resolveLocked(ctx, rp, &ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
if err != nil {
return nil, err
}
- // TODO(gvisor.dev/issue/1476): Implement BoundEndpointAt.
+ if err := d.checkPermissions(rp.Credentials(), vfs.MayWrite); err != nil {
+ return nil, err
+ }
+ if d.isSocket() {
+ if !d.isSynthetic() {
+ d.IncRef()
+ return &endpoint{
+ dentry: d,
+ file: d.file.file,
+ path: opts.Addr,
+ }, nil
+ }
+ return d.endpoint, nil
+ }
return nil, syserror.ECONNREFUSED
}
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index 8b4e91d17..9ab8fdc65 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -46,9 +46,11 @@ import (
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sentry/pgalloc"
+ "gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/unet"
@@ -70,7 +72,8 @@ type filesystem struct {
mfp pgalloc.MemoryFileProvider
// Immutable options.
- opts filesystemOptions
+ opts filesystemOptions
+ iopts InternalFilesystemOptions
// client is the client used by this filesystem. client is immutable.
client *p9.Client
@@ -207,6 +210,16 @@ const (
InteropModeShared
)
+// InternalFilesystemOptions may be passed as
+// vfs.GetFilesystemOptions.InternalData to FilesystemType.GetFilesystem.
+type InternalFilesystemOptions struct {
+ // If LeakConnection is true, do not close the connection to the server
+ // when the Filesystem is released. This is necessary for deployments in
+ // which servers can handle only a single client and report failure if that
+ // client disconnects.
+ LeakConnection bool
+}
+
// Name implements vfs.FilesystemType.Name.
func (FilesystemType) Name() string {
return Name
@@ -345,6 +358,14 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
return nil, nil, syserror.EINVAL
}
+ // Handle internal options.
+ iopts, ok := opts.InternalData.(InternalFilesystemOptions)
+ if opts.InternalData != nil && !ok {
+ ctx.Warningf("gofer.FilesystemType.GetFilesystem: GetFilesystemOptions.InternalData has type %T, wanted gofer.InternalFilesystemOptions", opts.InternalData)
+ return nil, nil, syserror.EINVAL
+ }
+ // If !ok, iopts being the zero value is correct.
+
// Establish a connection with the server.
conn, err := unet.NewSocket(fsopts.fd)
if err != nil {
@@ -381,6 +402,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
fs := &filesystem{
mfp: mfp,
opts: fsopts,
+ iopts: iopts,
uid: creds.EffectiveKUID,
gid: creds.EffectiveKGID,
client: client,
@@ -438,8 +460,10 @@ func (fs *filesystem) Release() {
// fs.
fs.syncMu.Unlock()
- // Close the connection to the server. This implicitly clunks all fids.
- fs.client.Close()
+ if !fs.iopts.LeakConnection {
+ // Close the connection to the server. This implicitly clunks all fids.
+ fs.client.Close()
+ }
}
// dentry implements vfs.DentryImpl.
@@ -472,10 +496,8 @@ type dentry struct {
// file is the unopened p9.File that backs this dentry. file is immutable.
//
// If file.isNil(), this dentry represents a synthetic file, i.e. a file
- // that does not exist on the remote filesystem. As of this writing, this
- // is only possible for a directory created with
- // MkdirOptions.ForSyntheticMountpoint == true.
- // TODO(gvisor.dev/issue/1476): Support synthetic sockets (and pipes).
+ // that does not exist on the remote filesystem. As of this writing, the
+ // only files that can be synthetic are sockets, pipes, and directories.
file p9file
// If deleted is non-zero, the file represented by this dentry has been
@@ -585,6 +607,14 @@ type dentry struct {
// and target are protected by dataMu.
haveTarget bool
target string
+
+ // If this dentry represents a synthetic socket file, endpoint is the
+ // transport endpoint bound to this file.
+ endpoint transport.BoundEndpoint
+
+ // If this dentry represents a synthetic named pipe, pipe is the pipe
+ // endpoint bound to this file.
+ pipe *pipe.VFSPipe
}
// dentryAttrMask returns a p9.AttrMask enabling all attributes used by the
@@ -631,11 +661,11 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma
},
}
d.pf.dentry = d
- if mask.UID && attr.UID != auth.NoID {
- d.uid = uint32(attr.UID)
+ if mask.UID {
+ d.uid = dentryUIDFromP9UID(attr.UID)
}
- if mask.GID && attr.GID != auth.NoID {
- d.gid = uint32(attr.GID)
+ if mask.GID {
+ d.gid = dentryGIDFromP9GID(attr.GID)
}
if mask.Size {
d.size = attr.Size
@@ -686,10 +716,10 @@ func (d *dentry) updateFromP9Attrs(mask p9.AttrMask, attr *p9.Attr) {
atomic.StoreUint32(&d.mode, uint32(attr.Mode))
}
if mask.UID {
- atomic.StoreUint32(&d.uid, uint32(attr.UID))
+ atomic.StoreUint32(&d.uid, dentryUIDFromP9UID(attr.UID))
}
if mask.GID {
- atomic.StoreUint32(&d.gid, uint32(attr.GID))
+ atomic.StoreUint32(&d.gid, dentryGIDFromP9GID(attr.GID))
}
// There is no P9_GETATTR_* bit for I/O block size.
if attr.BlockSize != 0 {
@@ -791,10 +821,21 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *lin
setLocalAtime = stat.Mask&linux.STATX_ATIME != 0
setLocalMtime = stat.Mask&linux.STATX_MTIME != 0
stat.Mask &^= linux.STATX_ATIME | linux.STATX_MTIME
- if !setLocalMtime && (stat.Mask&linux.STATX_SIZE != 0) {
- // Truncate updates mtime.
- setLocalMtime = true
- stat.Mtime.Nsec = linux.UTIME_NOW
+
+ // Prepare for truncate.
+ if stat.Mask&linux.STATX_SIZE != 0 {
+ switch d.mode & linux.S_IFMT {
+ case linux.S_IFREG:
+ if !setLocalMtime {
+ // Truncate updates mtime.
+ setLocalMtime = true
+ stat.Mtime.Nsec = linux.UTIME_NOW
+ }
+ case linux.S_IFDIR:
+ return syserror.EISDIR
+ default:
+ return syserror.EINVAL
+ }
}
}
d.metadataMu.Lock()
@@ -896,6 +937,20 @@ func (d *dentry) checkPermissions(creds *auth.Credentials, ats vfs.AccessTypes)
return vfs.GenericCheckPermissions(creds, ats, linux.FileMode(atomic.LoadUint32(&d.mode)), auth.KUID(atomic.LoadUint32(&d.uid)), auth.KGID(atomic.LoadUint32(&d.gid)))
}
+func dentryUIDFromP9UID(uid p9.UID) uint32 {
+ if !uid.Ok() {
+ return uint32(auth.OverflowUID)
+ }
+ return uint32(uid)
+}
+
+func dentryGIDFromP9GID(gid p9.GID) uint32 {
+ if !gid.Ok() {
+ return uint32(auth.OverflowGID)
+ }
+ return uint32(gid)
+}
+
// IncRef implements vfs.DentryImpl.IncRef.
func (d *dentry) IncRef() {
// d.refs may be 0 if d.fs.renameMu is locked, which serializes against
@@ -1049,6 +1104,7 @@ func (d *dentry) destroyLocked() {
d.handle.close(ctx)
}
d.handleMu.Unlock()
+
if !d.file.isNil() {
d.file.close(ctx)
d.file = p9file{}
@@ -1134,7 +1190,7 @@ func (d *dentry) removexattr(ctx context.Context, creds *auth.Credentials, name
return d.file.removeXattr(ctx, name)
}
-// Preconditions: !d.file.isNil(). d.isRegularFile() || d.isDirectory().
+// Preconditions: !d.isSynthetic(). d.isRegularFile() || d.isDirectory().
func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool) error {
// O_TRUNC unconditionally requires us to obtain a new handle (opened with
// O_TRUNC).
diff --git a/pkg/sentry/fsimpl/gofer/socket.go b/pkg/sentry/fsimpl/gofer/socket.go
new file mode 100644
index 000000000..d6dbe9092
--- /dev/null
+++ b/pkg/sentry/fsimpl/gofer/socket.go
@@ -0,0 +1,146 @@
+// 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 gofer
+
+import (
+ "syscall"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/p9"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
+ "gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
+ "gvisor.dev/gvisor/pkg/syserr"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+func (d *dentry) isSocket() bool {
+ return d.fileType() == linux.S_IFSOCK
+}
+
+// endpoint is a Gofer-backed transport.BoundEndpoint.
+//
+// An endpoint's lifetime is the time between when filesystem.BoundEndpointAt()
+// is called and either BoundEndpoint.BidirectionalConnect or
+// BoundEndpoint.UnidirectionalConnect is called.
+type endpoint struct {
+ // dentry is the filesystem dentry which produced this endpoint.
+ dentry *dentry
+
+ // file is the p9 file that contains a single unopened fid.
+ file p9.File
+
+ // path is the sentry path where this endpoint is bound.
+ path string
+}
+
+func sockTypeToP9(t linux.SockType) (p9.ConnectFlags, bool) {
+ switch t {
+ case linux.SOCK_STREAM:
+ return p9.StreamSocket, true
+ case linux.SOCK_SEQPACKET:
+ return p9.SeqpacketSocket, true
+ case linux.SOCK_DGRAM:
+ return p9.DgramSocket, true
+ }
+ return 0, false
+}
+
+// BidirectionalConnect implements ConnectableEndpoint.BidirectionalConnect.
+func (e *endpoint) BidirectionalConnect(ctx context.Context, ce transport.ConnectingEndpoint, returnConnect func(transport.Receiver, transport.ConnectedEndpoint)) *syserr.Error {
+ cf, ok := sockTypeToP9(ce.Type())
+ if !ok {
+ return syserr.ErrConnectionRefused
+ }
+
+ // No lock ordering required as only the ConnectingEndpoint has a mutex.
+ ce.Lock()
+
+ // Check connecting state.
+ if ce.Connected() {
+ ce.Unlock()
+ return syserr.ErrAlreadyConnected
+ }
+ if ce.Listening() {
+ ce.Unlock()
+ return syserr.ErrInvalidEndpointState
+ }
+
+ c, err := e.newConnectedEndpoint(ctx, cf, ce.WaiterQueue())
+ if err != nil {
+ ce.Unlock()
+ return err
+ }
+
+ returnConnect(c, c)
+ ce.Unlock()
+ if err := c.Init(); err != nil {
+ return syserr.FromError(err)
+ }
+
+ return nil
+}
+
+// UnidirectionalConnect implements
+// transport.BoundEndpoint.UnidirectionalConnect.
+func (e *endpoint) UnidirectionalConnect(ctx context.Context) (transport.ConnectedEndpoint, *syserr.Error) {
+ c, err := e.newConnectedEndpoint(ctx, p9.DgramSocket, &waiter.Queue{})
+ if err != nil {
+ return nil, err
+ }
+
+ if err := c.Init(); err != nil {
+ return nil, syserr.FromError(err)
+ }
+
+ // We don't need the receiver.
+ c.CloseRecv()
+ c.Release()
+
+ return c, nil
+}
+
+func (e *endpoint) newConnectedEndpoint(ctx context.Context, flags p9.ConnectFlags, queue *waiter.Queue) (*host.SCMConnectedEndpoint, *syserr.Error) {
+ hostFile, err := e.file.Connect(flags)
+ if err != nil {
+ return nil, syserr.ErrConnectionRefused
+ }
+ // Dup the fd so that the new endpoint can manage its lifetime.
+ hostFD, err := syscall.Dup(hostFile.FD())
+ if err != nil {
+ log.Warningf("Could not dup host socket fd %d: %v", hostFile.FD(), err)
+ return nil, syserr.FromError(err)
+ }
+ // After duplicating, we no longer need hostFile.
+ hostFile.Close()
+
+ c, serr := host.NewSCMEndpoint(ctx, hostFD, queue, e.path)
+ if serr != nil {
+ log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v flags %+v: %v", e.file, flags, serr)
+ return nil, serr
+ }
+ return c, nil
+}
+
+// Release implements transport.BoundEndpoint.Release.
+func (e *endpoint) Release() {
+ e.dentry.DecRef()
+}
+
+// Passcred implements transport.BoundEndpoint.Passcred.
+func (e *endpoint) Passcred() bool {
+ return false
+}
diff --git a/pkg/sentry/fsimpl/host/BUILD b/pkg/sentry/fsimpl/host/BUILD
index e1c56d89b..39509f703 100644
--- a/pkg/sentry/fsimpl/host/BUILD
+++ b/pkg/sentry/fsimpl/host/BUILD
@@ -20,6 +20,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/fdnotifier",
+ "//pkg/fspath",
"//pkg/log",
"//pkg/refs",
"//pkg/sentry/arch",
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index 05538ea42..144e04905 100644
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -24,6 +24,8 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fdnotifier"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
@@ -35,39 +37,12 @@ import (
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
)
-// filesystemType implements vfs.FilesystemType.
-type filesystemType struct{}
-
-// GetFilesystem implements FilesystemType.GetFilesystem.
-func (filesystemType) GetFilesystem(context.Context, *vfs.VirtualFilesystem, *auth.Credentials, string, vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
- panic("host.filesystemType.GetFilesystem should never be called")
-}
-
-// Name implements FilesystemType.Name.
-func (filesystemType) Name() string {
- return "none"
-}
-
-// filesystem implements vfs.FilesystemImpl.
-type filesystem struct {
- kernfs.Filesystem
-}
-
-// NewFilesystem sets up and returns a new hostfs filesystem.
-//
-// Note that there should only ever be one instance of host.filesystem,
-// a global mount for host fds.
-func NewFilesystem(vfsObj *vfs.VirtualFilesystem) *vfs.Filesystem {
- fs := &filesystem{}
- fs.Init(vfsObj, filesystemType{})
- return fs.VFSFilesystem()
-}
-
// ImportFD sets up and returns a vfs.FileDescription from a donated fd.
func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs.FileDescription, error) {
- fs, ok := mnt.Filesystem().Impl().(*kernfs.Filesystem)
+ fs, ok := mnt.Filesystem().Impl().(*filesystem)
if !ok {
return nil, fmt.Errorf("can't import host FDs into filesystems of type %T", mnt.Filesystem().Impl())
}
@@ -88,11 +63,12 @@ func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs
seekable := err != syserror.ESPIPE
i := &inode{
- hostFD: hostFD,
- seekable: seekable,
- isTTY: isTTY,
- canMap: canMap(uint32(fileType)),
- ino: fs.NextIno(),
+ hostFD: hostFD,
+ seekable: seekable,
+ isTTY: isTTY,
+ canMap: canMap(uint32(fileType)),
+ wouldBlock: wouldBlock(uint32(fileType)),
+ ino: fs.NextIno(),
// For simplicity, set offset to 0. Technically, we should use the existing
// offset on the host if the file is seekable.
offset: 0,
@@ -103,14 +79,60 @@ func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs
panic("files that can return EWOULDBLOCK (sockets, pipes, etc.) cannot be memory mapped")
}
+ // If the hostFD would block, we must set it to non-blocking and handle
+ // blocking behavior in the sentry.
+ if i.wouldBlock {
+ if err := syscall.SetNonblock(i.hostFD, true); err != nil {
+ return nil, err
+ }
+ if err := fdnotifier.AddFD(int32(i.hostFD), &i.queue); err != nil {
+ return nil, err
+ }
+ }
+
d := &kernfs.Dentry{}
d.Init(i)
+
// i.open will take a reference on d.
defer d.DecRef()
-
return i.open(ctx, d.VFSDentry(), mnt)
}
+// filesystemType implements vfs.FilesystemType.
+type filesystemType struct{}
+
+// GetFilesystem implements FilesystemType.GetFilesystem.
+func (filesystemType) GetFilesystem(context.Context, *vfs.VirtualFilesystem, *auth.Credentials, string, vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
+ panic("host.filesystemType.GetFilesystem should never be called")
+}
+
+// Name implements FilesystemType.Name.
+func (filesystemType) Name() string {
+ return "none"
+}
+
+// NewFilesystem sets up and returns a new hostfs filesystem.
+//
+// Note that there should only ever be one instance of host.filesystem,
+// a global mount for host fds.
+func NewFilesystem(vfsObj *vfs.VirtualFilesystem) *vfs.Filesystem {
+ fs := &filesystem{}
+ fs.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
+ return fs.VFSFilesystem()
+}
+
+// filesystem implements vfs.FilesystemImpl.
+type filesystem struct {
+ kernfs.Filesystem
+}
+
+func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDentry, b *fspath.Builder) error {
+ d := vd.Dentry().Impl().(*kernfs.Dentry)
+ inode := d.Inode().(*inode)
+ b.PrependComponent(fmt.Sprintf("host:[%d]", inode.ino))
+ return vfs.PrependPathSyntheticError{}
+}
+
// inode implements kernfs.Inode.
type inode struct {
kernfs.InodeNotDirectory
@@ -125,6 +147,12 @@ type inode struct {
// This field is initialized at creation time and is immutable.
hostFD int
+ // wouldBlock is true if the host FD would return EWOULDBLOCK for
+ // operations that would block.
+ //
+ // This field is initialized at creation time and is immutable.
+ wouldBlock bool
+
// seekable is false if the host fd points to a file representing a stream,
// e.g. a socket or a pipe. Such files are not seekable and can return
// EWOULDBLOCK for I/O operations.
@@ -152,6 +180,9 @@ type inode struct {
// offset specifies the current file offset.
offset int64
+
+ // Event queue for blocking operations.
+ queue waiter.Queue
}
// Note that these flags may become out of date, since they can be modified
@@ -354,6 +385,9 @@ func (i *inode) DecRef() {
// Destroy implements kernfs.Inode.
func (i *inode) Destroy() {
+ if i.wouldBlock {
+ fdnotifier.RemoveFD(int32(i.hostFD))
+ }
if err := unix.Close(i.hostFD); err != nil {
log.Warningf("failed to close host fd %d: %v", i.hostFD, err)
}
@@ -385,7 +419,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
}
@@ -396,7 +430,7 @@ func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount) (*vfs.F
// TODO(gvisor.dev/issue/1672): Whitelist specific file types here, so that
// we don't allow importing arbitrary file types without proper support.
if i.isTTY {
- fd := &ttyFD{
+ fd := &TTYFileDescription{
fileDescription: fileDescription{inode: i},
termios: linux.DefaultSlaveTermios,
}
@@ -614,3 +648,20 @@ func (f *fileDescription) ConfigureMMap(_ context.Context, opts *memmap.MMapOpts
// TODO(gvisor.dev/issue/1672): Implement ConfigureMMap and Mappable interface.
return syserror.ENODEV
}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (f *fileDescription) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ f.inode.queue.EventRegister(e, mask)
+ fdnotifier.UpdateFD(int32(f.inode.hostFD))
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (f *fileDescription) EventUnregister(e *waiter.Entry) {
+ f.inode.queue.EventUnregister(e)
+ fdnotifier.UpdateFD(int32(f.inode.hostFD))
+}
+
+// Readiness uses the poll() syscall to check the status of the underlying FD.
+func (f *fileDescription) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return fdnotifier.NonBlockingPoll(int32(f.inode.hostFD), mask)
+}
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/host/tty.go b/pkg/sentry/fsimpl/host/tty.go
index 8936afb06..68af6e5af 100644
--- a/pkg/sentry/fsimpl/host/tty.go
+++ b/pkg/sentry/fsimpl/host/tty.go
@@ -26,15 +26,15 @@ import (
"gvisor.dev/gvisor/pkg/usermem"
)
-// ttyFD implements vfs.FileDescriptionImpl for a host file descriptor
-// that wraps a TTY FD.
-type ttyFD struct {
+// TTYFileDescription implements vfs.FileDescriptionImpl for a host file
+// descriptor that wraps a TTY FD.
+type TTYFileDescription struct {
fileDescription
// mu protects the fields below.
mu sync.Mutex `state:"nosave"`
- // session is the session attached to this ttyFD.
+ // session is the session attached to this TTYFileDescription.
session *kernel.Session
// fgProcessGroup is the foreground process group that is currently
@@ -48,7 +48,7 @@ type ttyFD struct {
// InitForegroundProcessGroup sets the foreground process group and session for
// the TTY. This should only be called once, after the foreground process group
// has been created, but before it has started running.
-func (t *ttyFD) InitForegroundProcessGroup(pg *kernel.ProcessGroup) {
+func (t *TTYFileDescription) InitForegroundProcessGroup(pg *kernel.ProcessGroup) {
t.mu.Lock()
defer t.mu.Unlock()
if t.fgProcessGroup != nil {
@@ -59,14 +59,14 @@ func (t *ttyFD) InitForegroundProcessGroup(pg *kernel.ProcessGroup) {
}
// ForegroundProcessGroup returns the foreground process for the TTY.
-func (t *ttyFD) ForegroundProcessGroup() *kernel.ProcessGroup {
+func (t *TTYFileDescription) ForegroundProcessGroup() *kernel.ProcessGroup {
t.mu.Lock()
defer t.mu.Unlock()
return t.fgProcessGroup
}
// Release implements fs.FileOperations.Release.
-func (t *ttyFD) Release() {
+func (t *TTYFileDescription) Release() {
t.mu.Lock()
t.fgProcessGroup = nil
t.mu.Unlock()
@@ -78,7 +78,7 @@ func (t *ttyFD) Release() {
//
// Reading from a TTY is only allowed for foreground process groups. Background
// process groups will either get EIO or a SIGTTIN.
-func (t *ttyFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
+func (t *TTYFileDescription) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -96,7 +96,7 @@ func (t *ttyFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64,
//
// Reading from a TTY is only allowed for foreground process groups. Background
// process groups will either get EIO or a SIGTTIN.
-func (t *ttyFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+func (t *TTYFileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -111,7 +111,7 @@ func (t *ttyFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadO
}
// PWrite implements vfs.FileDescriptionImpl.
-func (t *ttyFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
+func (t *TTYFileDescription) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -126,7 +126,7 @@ func (t *ttyFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64
}
// Write implements vfs.FileDescriptionImpl.
-func (t *ttyFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+func (t *TTYFileDescription) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -141,7 +141,7 @@ func (t *ttyFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.Writ
}
// Ioctl implements vfs.FileDescriptionImpl.
-func (t *ttyFD) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+func (t *TTYFileDescription) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
// Ignore arg[0]. This is the real FD:
fd := t.inode.hostFD
ioctl := args[1].Uint64()
@@ -321,7 +321,7 @@ func (t *ttyFD) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArgum
// is a bit convoluted, but documented inline.
//
// Preconditions: t.mu must be held.
-func (t *ttyFD) checkChange(ctx context.Context, sig linux.Signal) error {
+func (t *TTYFileDescription) checkChange(ctx context.Context, sig linux.Signal) error {
task := kernel.TaskFromContext(ctx)
if task == nil {
// No task? Linux does not have an analog for this case, but
diff --git a/pkg/sentry/fsimpl/kernfs/fd_impl_util.go b/pkg/sentry/fsimpl/kernfs/fd_impl_util.go
index dd5806301..8284e76a7 100644
--- a/pkg/sentry/fsimpl/kernfs/fd_impl_util.go
+++ b/pkg/sentry/fsimpl/kernfs/fd_impl_util.go
@@ -22,6 +22,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -36,13 +37,20 @@ import (
// inode.
//
// Must be initialize with Init before first use.
+//
+// Lock ordering: mu => children.mu.
type GenericDirectoryFD struct {
vfs.FileDescriptionDefaultImpl
vfs.DirectoryFileDescriptionDefaultImpl
vfsfd vfs.FileDescription
children *OrderedChildren
- off int64
+
+ // mu protects the fields below.
+ mu sync.Mutex
+
+ // off is the current directory offset. Protected by "mu".
+ off int64
}
// NewGenericDirectoryFD creates a new GenericDirectoryFD and returns its
@@ -115,17 +123,13 @@ func (fd *GenericDirectoryFD) inode() Inode {
// IterDirents implements vfs.FileDecriptionImpl.IterDirents. IterDirents holds
// o.mu when calling cb.
func (fd *GenericDirectoryFD) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback) error {
- vfsFS := fd.filesystem()
- fs := vfsFS.Impl().(*Filesystem)
- vfsd := fd.vfsfd.VirtualDentry().Dentry()
-
- fs.mu.Lock()
- defer fs.mu.Unlock()
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
opts := vfs.StatOptions{Mask: linux.STATX_INO}
// Handle ".".
if fd.off == 0 {
- stat, err := fd.inode().Stat(vfsFS, opts)
+ stat, err := fd.inode().Stat(fd.filesystem(), opts)
if err != nil {
return err
}
@@ -143,8 +147,9 @@ func (fd *GenericDirectoryFD) IterDirents(ctx context.Context, cb vfs.IterDirent
// Handle "..".
if fd.off == 1 {
+ vfsd := fd.vfsfd.VirtualDentry().Dentry()
parentInode := genericParentOrSelf(vfsd.Impl().(*Dentry)).inode
- stat, err := parentInode.Stat(vfsFS, opts)
+ stat, err := parentInode.Stat(fd.filesystem(), opts)
if err != nil {
return err
}
@@ -168,7 +173,7 @@ func (fd *GenericDirectoryFD) IterDirents(ctx context.Context, cb vfs.IterDirent
childIdx := fd.off - 2
for it := fd.children.nthLocked(childIdx); it != nil; it = it.Next() {
inode := it.Dentry.Impl().(*Dentry).inode
- stat, err := inode.Stat(vfsFS, opts)
+ stat, err := inode.Stat(fd.filesystem(), opts)
if err != nil {
return err
}
@@ -192,9 +197,8 @@ func (fd *GenericDirectoryFD) IterDirents(ctx context.Context, cb vfs.IterDirent
// Seek implements vfs.FileDecriptionImpl.Seek.
func (fd *GenericDirectoryFD) Seek(ctx context.Context, offset int64, whence int32) (int64, error) {
- fs := fd.filesystem().Impl().(*Filesystem)
- fs.mu.Lock()
- defer fs.mu.Unlock()
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
switch whence {
case linux.SEEK_SET:
diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go
index 9e8d80414..4a12ae245 100644
--- a/pkg/sentry/fsimpl/kernfs/filesystem.go
+++ b/pkg/sentry/fsimpl/kernfs/filesystem.go
@@ -766,14 +766,17 @@ func (fs *Filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
-func (fs *Filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
+func (fs *Filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
fs.mu.RLock()
- _, _, err := fs.walkExistingLocked(ctx, rp)
+ _, inode, err := fs.walkExistingLocked(ctx, rp)
fs.mu.RUnlock()
fs.processDeferredDecRefs()
if err != nil {
return nil, err
}
+ if err := inode.CheckPermissions(ctx, rp.Credentials(), vfs.MayWrite); err != nil {
+ return nil, err
+ }
return nil, syserror.ECONNREFUSED
}
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go
index 732837933..a83151ad3 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs.go
@@ -132,13 +132,6 @@ func (fs *Filesystem) processDeferredDecRefsLocked() {
fs.droppedDentriesMu.Unlock()
}
-// Init initializes a kernfs filesystem. This should be called from during
-// vfs.FilesystemType.NewFilesystem for the concrete filesystem embedding
-// kernfs.
-func (fs *Filesystem) Init(vfsObj *vfs.VirtualFilesystem, fsType vfs.FilesystemType) {
- fs.vfsfs.Init(vfsObj, fsType, fs)
-}
-
// VFSFilesystem returns the generic vfs filesystem object.
func (fs *Filesystem) VFSFilesystem() *vfs.Filesystem {
return &fs.vfsfs
@@ -261,6 +254,11 @@ func (d *Dentry) insertChildLocked(name string, child *Dentry) {
d.children[name] = child
}
+// Inode returns the dentry's inode.
+func (d *Dentry) Inode() Inode {
+ return d.inode
+}
+
// The Inode interface maps filesystem-level operations that operate on paths to
// equivalent operations on specific filesystem nodes.
//
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs_test.go b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
index a9f671bc8..1c5d3e7e7 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs_test.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
@@ -195,7 +195,7 @@ func (fsType) Name() string {
func (fst fsType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opt vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
fs := &filesystem{}
- fs.Init(vfsObj, &fst)
+ fs.VFSFilesystem().Init(vfsObj, &fst, fs)
root := fst.rootFn(creds, fs)
return fs.VFSFilesystem(), root.VFSDentry(), nil
}
diff --git a/pkg/sentry/fsimpl/pipefs/pipefs.go b/pkg/sentry/fsimpl/pipefs/pipefs.go
index d6bd67467..5375e5e75 100644
--- a/pkg/sentry/fsimpl/pipefs/pipefs.go
+++ b/pkg/sentry/fsimpl/pipefs/pipefs.go
@@ -40,25 +40,19 @@ func (filesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFile
panic("pipefs.filesystemType.GetFilesystem should never be called")
}
-// filesystem implements vfs.FilesystemImpl.
-type filesystem struct {
- kernfs.Filesystem
-
- // TODO(gvisor.dev/issue/1193):
- //
- // - kernfs does not provide a way to implement statfs, from which we
- // should indicate PIPEFS_MAGIC.
- //
- // - kernfs does not provide a way to override names for
- // vfs.FilesystemImpl.PrependPath(); pipefs inodes should use synthetic
- // name fmt.Sprintf("pipe:[%d]", inode.ino).
-}
+// TODO(gvisor.dev/issue/1193):
+//
+// - kernfs does not provide a way to implement statfs, from which we
+// should indicate PIPEFS_MAGIC.
+//
+// - kernfs does not provide a way to override names for
+// vfs.FilesystemImpl.PrependPath(); pipefs inodes should use synthetic
+// name fmt.Sprintf("pipe:[%d]", inode.ino).
-// NewFilesystem sets up and returns a new vfs.Filesystem implemented by
-// pipefs.
+// NewFilesystem sets up and returns a new vfs.Filesystem implemented by pipefs.
func NewFilesystem(vfsObj *vfs.VirtualFilesystem) *vfs.Filesystem {
- fs := &filesystem{}
- fs.Init(vfsObj, filesystemType{})
+ fs := &kernfs.Filesystem{}
+ fs.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
return fs.VFSFilesystem()
}
diff --git a/pkg/sentry/fsimpl/proc/task_net.go b/pkg/sentry/fsimpl/proc/task_net.go
index 6595fcee6..9c329341a 100644
--- a/pkg/sentry/fsimpl/proc/task_net.go
+++ b/pkg/sentry/fsimpl/proc/task_net.go
@@ -511,6 +511,8 @@ func (d *netUDPData) Generate(ctx context.Context, buf *bytes.Buffer) error {
// degrade gracefully and retrieve what we can.
t := kernel.TaskFromContext(ctx)
+ buf.WriteString(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops \n")
+
for _, se := range d.kernel.ListSockets() {
s := se.SockVFS2
if !s.TryIncRef() {
diff --git a/pkg/sentry/fsimpl/proc/tasks_files.go b/pkg/sentry/fsimpl/proc/tasks_files.go
index 92007df81..e5f13b69e 100644
--- a/pkg/sentry/fsimpl/proc/tasks_files.go
+++ b/pkg/sentry/fsimpl/proc/tasks_files.go
@@ -272,12 +272,16 @@ func (*meminfoData) Generate(ctx context.Context, buf *bytes.Buffer) error {
inactiveFile := file - activeFile
fmt.Fprintf(buf, "MemTotal: %8d kB\n", totalSize/1024)
- memFree := (totalSize - totalUsage) / 1024
+ memFree := totalSize - totalUsage
+ if memFree > totalSize {
+ // Underflow.
+ memFree = 0
+ }
// We use MemFree as MemAvailable because we don't swap.
// TODO(rahat): When reclaim is implemented the value of MemAvailable
// should change.
- fmt.Fprintf(buf, "MemFree: %8d kB\n", memFree)
- fmt.Fprintf(buf, "MemAvailable: %8d kB\n", memFree)
+ fmt.Fprintf(buf, "MemFree: %8d kB\n", memFree/1024)
+ fmt.Fprintf(buf, "MemAvailable: %8d kB\n", memFree/1024)
fmt.Fprintf(buf, "Buffers: 0 kB\n") // memory usage by block devices
fmt.Fprintf(buf, "Cached: %8d kB\n", (file+snapshot.Tmpfs)/1024)
// Emulate a system with no swap, which disables inactivation of anon pages.
diff --git a/pkg/sentry/fsimpl/proc/tasks_sys.go b/pkg/sentry/fsimpl/proc/tasks_sys.go
index f08668ca2..0e90e02fe 100644
--- a/pkg/sentry/fsimpl/proc/tasks_sys.go
+++ b/pkg/sentry/fsimpl/proc/tasks_sys.go
@@ -112,9 +112,7 @@ func newSysNetDir(root *auth.Credentials, inoGen InoGenerator, k *kernel.Kernel)
}
}
- return kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, map[string]*kernfs.Dentry{
- "net": kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, contents),
- })
+ return kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, contents)
}
// mmapMinAddrData implements vfs.DynamicBytesSource for
diff --git a/pkg/sentry/fsimpl/sockfs/sockfs.go b/pkg/sentry/fsimpl/sockfs/sockfs.go
index 271134af8..239a9f4b4 100644
--- a/pkg/sentry/fsimpl/sockfs/sockfs.go
+++ b/pkg/sentry/fsimpl/sockfs/sockfs.go
@@ -41,26 +41,19 @@ func (filesystemType) Name() string {
return "sockfs"
}
-// filesystem implements vfs.FilesystemImpl.
-type filesystem struct {
- kernfs.Filesystem
-}
-
// NewFilesystem sets up and returns a new sockfs filesystem.
//
// Note that there should only ever be one instance of sockfs.Filesystem,
// backing a global socket mount.
func NewFilesystem(vfsObj *vfs.VirtualFilesystem) *vfs.Filesystem {
- fs := &filesystem{}
- fs.Init(vfsObj, filesystemType{})
+ fs := &kernfs.Filesystem{}
+ fs.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
return fs.VFSFilesystem()
}
// inode implements kernfs.Inode.
//
-// TODO(gvisor.dev/issue/1476): Add device numbers to this inode (which are
-// not included in InodeAttrs) to store the numbers of the appropriate
-// socket device. Override InodeAttrs.Stat() accordingly.
+// TODO(gvisor.dev/issue/1193): Device numbers.
type inode struct {
kernfs.InodeNotDirectory
kernfs.InodeNotSymlink
diff --git a/pkg/sentry/fsimpl/sys/sys.go b/pkg/sentry/fsimpl/sys/sys.go
index f8d25d35e..00f7d6214 100644
--- a/pkg/sentry/fsimpl/sys/sys.go
+++ b/pkg/sentry/fsimpl/sys/sys.go
@@ -47,7 +47,7 @@ func (FilesystemType) Name() string {
// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
fs := &filesystem{}
- fs.Filesystem.Init(vfsObj, &fsType)
+ fs.VFSFilesystem().Init(vfsObj, &fsType, fs)
k := kernel.KernelFromContext(ctx)
maxCPUCores := k.ApplicationCores()
defaultSysDirMode := linux.FileMode(0755)
diff --git a/pkg/sentry/fsimpl/tmpfs/filesystem.go b/pkg/sentry/fsimpl/tmpfs/filesystem.go
index 5b62f9ebb..36ffcb592 100644
--- a/pkg/sentry/fsimpl/tmpfs/filesystem.go
+++ b/pkg/sentry/fsimpl/tmpfs/filesystem.go
@@ -697,13 +697,16 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
-func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
+func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
fs.mu.RLock()
defer fs.mu.RUnlock()
d, err := resolveLocked(rp)
if err != nil {
return nil, err
}
+ if err := d.inode.checkPermissions(rp.Credentials(), vfs.MayWrite); err != nil {
+ return nil, err
+ }
switch impl := d.inode.impl.(type) {
case *socketFile:
return impl.ep, nil