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.go40
-rw-r--r--pkg/sentry/fsimpl/eventfd/BUILD33
-rw-r--r--pkg/sentry/fsimpl/eventfd/eventfd.go284
-rw-r--r--pkg/sentry/fsimpl/eventfd/eventfd_test.go97
-rw-r--r--pkg/sentry/fsimpl/ext/ext.go16
-rw-r--r--pkg/sentry/fsimpl/ext/filesystem.go8
-rw-r--r--pkg/sentry/fsimpl/ext/inode.go2
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go62
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go23
-rw-r--r--pkg/sentry/fsimpl/gofer/special_file.go28
-rw-r--r--pkg/sentry/fsimpl/host/host.go205
-rw-r--r--pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go4
-rw-r--r--pkg/sentry/fsimpl/kernfs/inode_impl_util.go36
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs_test.go6
-rw-r--r--pkg/sentry/fsimpl/kernfs/symlink.go8
-rw-r--r--pkg/sentry/fsimpl/pipefs/BUILD1
-rw-r--r--pkg/sentry/fsimpl/pipefs/pipefs.go79
-rw-r--r--pkg/sentry/fsimpl/proc/filesystem.go29
-rw-r--r--pkg/sentry/fsimpl/proc/subtasks.go12
-rw-r--r--pkg/sentry/fsimpl/proc/task.go64
-rw-r--r--pkg/sentry/fsimpl/proc/task_fds.go28
-rw-r--r--pkg/sentry/fsimpl/proc/task_files.go16
-rw-r--r--pkg/sentry/fsimpl/proc/task_net.go38
-rw-r--r--pkg/sentry/fsimpl/proc/tasks.go47
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_files.go8
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_sys.go108
-rw-r--r--pkg/sentry/fsimpl/signalfd/BUILD20
-rw-r--r--pkg/sentry/fsimpl/signalfd/signalfd.go135
-rw-r--r--pkg/sentry/fsimpl/sockfs/BUILD1
-rw-r--r--pkg/sentry/fsimpl/sockfs/sockfs.go46
-rw-r--r--pkg/sentry/fsimpl/sys/sys.go21
-rw-r--r--pkg/sentry/fsimpl/timerfd/BUILD17
-rw-r--r--pkg/sentry/fsimpl/timerfd/timerfd.go143
-rw-r--r--pkg/sentry/fsimpl/tmpfs/tmpfs.go39
34 files changed, 1331 insertions, 373 deletions
diff --git a/pkg/sentry/fsimpl/devpts/devpts.go b/pkg/sentry/fsimpl/devpts/devpts.go
index 94db8fe5c..c03c65445 100644
--- a/pkg/sentry/fsimpl/devpts/devpts.go
+++ b/pkg/sentry/fsimpl/devpts/devpts.go
@@ -51,21 +51,37 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
return nil, nil, syserror.EINVAL
}
- fs, root := fstype.newFilesystem(vfsObj, creds)
- return fs.VFSFilesystem(), root.VFSDentry(), nil
+ fs, root, err := fstype.newFilesystem(vfsObj, creds)
+ if err != nil {
+ return nil, nil, err
+ }
+ return fs.Filesystem.VFSFilesystem(), root.VFSDentry(), nil
+}
+
+type filesystem struct {
+ kernfs.Filesystem
+
+ devMinor uint32
}
// newFilesystem creates a new devpts filesystem with root directory and ptmx
// 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.VFSFilesystem().Init(vfsObj, fstype, fs)
+func (fstype FilesystemType) newFilesystem(vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials) (*filesystem, *kernfs.Dentry, error) {
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+
+ fs := &filesystem{
+ devMinor: devMinor,
+ }
+ fs.Filesystem.VFSFilesystem().Init(vfsObj, fstype, fs)
// Construct the root directory. This is always inode id 1.
root := &rootInode{
slaves: make(map[uint32]*slaveInode),
}
- root.InodeAttrs.Init(creds, 1, linux.ModeDirectory|0555)
+ root.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, devMinor, 1, linux.ModeDirectory|0555)
root.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
root.dentry.Init(root)
@@ -74,7 +90,7 @@ func (fstype FilesystemType) newFilesystem(vfsObj *vfs.VirtualFilesystem, creds
master := &masterInode{
root: root,
}
- master.InodeAttrs.Init(creds, 2, linux.ModeCharacterDevice|0666)
+ master.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, devMinor, 2, linux.ModeCharacterDevice|0666)
master.dentry.Init(master)
// Add the master as a child of the root.
@@ -83,7 +99,13 @@ func (fstype FilesystemType) newFilesystem(vfsObj *vfs.VirtualFilesystem, creds
})
root.IncLinks(links)
- return fs, &root.dentry
+ return fs, &root.dentry, nil
+}
+
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release() {
+ fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release()
}
// rootInode is the root directory inode for the devpts mounts.
@@ -140,7 +162,7 @@ func (i *rootInode) allocateTerminal(creds *auth.Credentials) (*Terminal, error)
}
// Linux always uses pty index + 3 as the inode id. See
// fs/devpts/inode.c:devpts_pty_new().
- slave.InodeAttrs.Init(creds, uint64(idx+3), linux.ModeCharacterDevice|0600)
+ slave.InodeAttrs.Init(creds, i.InodeAttrs.DevMajor(), i.InodeAttrs.DevMinor(), uint64(idx+3), linux.ModeCharacterDevice|0600)
slave.dentry.Init(slave)
i.slaves[idx] = slave
diff --git a/pkg/sentry/fsimpl/eventfd/BUILD b/pkg/sentry/fsimpl/eventfd/BUILD
new file mode 100644
index 000000000..ea167d38c
--- /dev/null
+++ b/pkg/sentry/fsimpl/eventfd/BUILD
@@ -0,0 +1,33 @@
+load("//tools:defs.bzl", "go_library", "go_test")
+
+licenses(["notice"])
+
+go_library(
+ name = "eventfd",
+ srcs = ["eventfd.go"],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/fdnotifier",
+ "//pkg/log",
+ "//pkg/sentry/vfs",
+ "//pkg/syserror",
+ "//pkg/usermem",
+ "//pkg/waiter",
+ ],
+)
+
+go_test(
+ name = "eventfd_test",
+ size = "small",
+ srcs = ["eventfd_test.go"],
+ library = ":eventfd",
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/sentry/contexttest",
+ "//pkg/sentry/vfs",
+ "//pkg/usermem",
+ "//pkg/waiter",
+ ],
+)
diff --git a/pkg/sentry/fsimpl/eventfd/eventfd.go b/pkg/sentry/fsimpl/eventfd/eventfd.go
new file mode 100644
index 000000000..c573d7935
--- /dev/null
+++ b/pkg/sentry/fsimpl/eventfd/eventfd.go
@@ -0,0 +1,284 @@
+// 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 eventfd implements event fds.
+package eventfd
+
+import (
+ "math"
+ "sync"
+ "syscall"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fdnotifier"
+ "gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// EventFileDescription implements FileDescriptionImpl for file-based event
+// notification (eventfd). Eventfds are usually internal to the Sentry but in
+// certain situations they may be converted into a host-backed eventfd.
+type EventFileDescription struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ // queue is used to notify interested parties when the event object
+ // becomes readable or writable.
+ queue waiter.Queue `state:"zerovalue"`
+
+ // mu protects the fields below.
+ mu sync.Mutex `state:"nosave"`
+
+ // val is the current value of the event counter.
+ val uint64
+
+ // semMode specifies whether the event is in "semaphore" mode.
+ semMode bool
+
+ // hostfd indicates whether this eventfd is passed through to the host.
+ hostfd int
+}
+
+var _ vfs.FileDescriptionImpl = (*EventFileDescription)(nil)
+
+// New creates a new event fd.
+func New(vfsObj *vfs.VirtualFilesystem, initVal uint64, semMode bool, flags uint32) (*vfs.FileDescription, error) {
+ vd := vfsObj.NewAnonVirtualDentry("[eventfd]")
+ defer vd.DecRef()
+ efd := &EventFileDescription{
+ val: initVal,
+ semMode: semMode,
+ hostfd: -1,
+ }
+ if err := efd.vfsfd.Init(efd, flags, vd.Mount(), vd.Dentry(), &vfs.FileDescriptionOptions{
+ UseDentryMetadata: true,
+ DenyPRead: true,
+ DenyPWrite: true,
+ }); err != nil {
+ return nil, err
+ }
+ return &efd.vfsfd, nil
+}
+
+// HostFD returns the host eventfd associated with this event.
+func (efd *EventFileDescription) HostFD() (int, error) {
+ efd.mu.Lock()
+ defer efd.mu.Unlock()
+ if efd.hostfd >= 0 {
+ return efd.hostfd, nil
+ }
+
+ flags := linux.EFD_NONBLOCK
+ if efd.semMode {
+ flags |= linux.EFD_SEMAPHORE
+ }
+
+ fd, _, errno := syscall.Syscall(syscall.SYS_EVENTFD2, uintptr(efd.val), uintptr(flags), 0)
+ if errno != 0 {
+ return -1, errno
+ }
+
+ if err := fdnotifier.AddFD(int32(fd), &efd.queue); err != nil {
+ if closeErr := syscall.Close(int(fd)); closeErr != nil {
+ log.Warningf("close(%d) eventfd failed: %v", fd, closeErr)
+ }
+ return -1, err
+ }
+
+ efd.hostfd = int(fd)
+ return efd.hostfd, nil
+}
+
+// Release implements FileDescriptionImpl.Release()
+func (efd *EventFileDescription) Release() {
+ efd.mu.Lock()
+ defer efd.mu.Unlock()
+ if efd.hostfd >= 0 {
+ fdnotifier.RemoveFD(int32(efd.hostfd))
+ if closeErr := syscall.Close(int(efd.hostfd)); closeErr != nil {
+ log.Warningf("close(%d) eventfd failed: %v", efd.hostfd, closeErr)
+ }
+ efd.hostfd = -1
+ }
+}
+
+// Read implements FileDescriptionImpl.Read.
+func (efd *EventFileDescription) Read(ctx context.Context, dst usermem.IOSequence, _ vfs.ReadOptions) (int64, error) {
+ if dst.NumBytes() < 8 {
+ return 0, syscall.EINVAL
+ }
+ if err := efd.read(ctx, dst); err != nil {
+ return 0, err
+ }
+ return 8, nil
+}
+
+// Write implements FileDescriptionImpl.Write.
+func (efd *EventFileDescription) Write(ctx context.Context, src usermem.IOSequence, _ vfs.WriteOptions) (int64, error) {
+ if src.NumBytes() < 8 {
+ return 0, syscall.EINVAL
+ }
+ if err := efd.write(ctx, src); err != nil {
+ return 0, err
+ }
+ return 8, nil
+}
+
+// Preconditions: Must be called with efd.mu locked.
+func (efd *EventFileDescription) hostReadLocked(ctx context.Context, dst usermem.IOSequence) error {
+ var buf [8]byte
+ if _, err := syscall.Read(efd.hostfd, buf[:]); err != nil {
+ if err == syscall.EWOULDBLOCK {
+ return syserror.ErrWouldBlock
+ }
+ return err
+ }
+ _, err := dst.CopyOut(ctx, buf[:])
+ return err
+}
+
+func (efd *EventFileDescription) read(ctx context.Context, dst usermem.IOSequence) error {
+ efd.mu.Lock()
+ if efd.hostfd >= 0 {
+ defer efd.mu.Unlock()
+ return efd.hostReadLocked(ctx, dst)
+ }
+
+ // We can't complete the read if the value is currently zero.
+ if efd.val == 0 {
+ efd.mu.Unlock()
+ return syserror.ErrWouldBlock
+ }
+
+ // Update the value based on the mode the event is operating in.
+ var val uint64
+ if efd.semMode {
+ val = 1
+ // Consistent with Linux, this is done even if writing to memory fails.
+ efd.val--
+ } else {
+ val = efd.val
+ efd.val = 0
+ }
+
+ efd.mu.Unlock()
+
+ // Notify writers. We do this even if we were already writable because
+ // it is possible that a writer is waiting to write the maximum value
+ // to the event.
+ efd.queue.Notify(waiter.EventOut)
+
+ var buf [8]byte
+ usermem.ByteOrder.PutUint64(buf[:], val)
+ _, err := dst.CopyOut(ctx, buf[:])
+ return err
+}
+
+// Preconditions: Must be called with efd.mu locked.
+func (efd *EventFileDescription) hostWriteLocked(val uint64) error {
+ var buf [8]byte
+ usermem.ByteOrder.PutUint64(buf[:], val)
+ _, err := syscall.Write(efd.hostfd, buf[:])
+ if err == syscall.EWOULDBLOCK {
+ return syserror.ErrWouldBlock
+ }
+ return err
+}
+
+func (efd *EventFileDescription) write(ctx context.Context, src usermem.IOSequence) error {
+ var buf [8]byte
+ if _, err := src.CopyIn(ctx, buf[:]); err != nil {
+ return err
+ }
+ val := usermem.ByteOrder.Uint64(buf[:])
+
+ return efd.Signal(val)
+}
+
+// Signal is an internal function to signal the event fd.
+func (efd *EventFileDescription) Signal(val uint64) error {
+ if val == math.MaxUint64 {
+ return syscall.EINVAL
+ }
+
+ efd.mu.Lock()
+
+ if efd.hostfd >= 0 {
+ defer efd.mu.Unlock()
+ return efd.hostWriteLocked(val)
+ }
+
+ // We only allow writes that won't cause the value to go over the max
+ // uint64 minus 1.
+ if val > math.MaxUint64-1-efd.val {
+ efd.mu.Unlock()
+ return syserror.ErrWouldBlock
+ }
+
+ efd.val += val
+ efd.mu.Unlock()
+
+ // Always trigger a notification.
+ efd.queue.Notify(waiter.EventIn)
+
+ return nil
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (efd *EventFileDescription) Readiness(mask waiter.EventMask) waiter.EventMask {
+ efd.mu.Lock()
+ defer efd.mu.Unlock()
+
+ if efd.hostfd >= 0 {
+ return fdnotifier.NonBlockingPoll(int32(efd.hostfd), mask)
+ }
+
+ ready := waiter.EventMask(0)
+ if efd.val > 0 {
+ ready |= waiter.EventIn
+ }
+
+ if efd.val < math.MaxUint64-1 {
+ ready |= waiter.EventOut
+ }
+
+ return mask & ready
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (efd *EventFileDescription) EventRegister(entry *waiter.Entry, mask waiter.EventMask) {
+ efd.queue.EventRegister(entry, mask)
+
+ efd.mu.Lock()
+ defer efd.mu.Unlock()
+ if efd.hostfd >= 0 {
+ fdnotifier.UpdateFD(int32(efd.hostfd))
+ }
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (efd *EventFileDescription) EventUnregister(entry *waiter.Entry) {
+ efd.queue.EventUnregister(entry)
+
+ efd.mu.Lock()
+ defer efd.mu.Unlock()
+ if efd.hostfd >= 0 {
+ fdnotifier.UpdateFD(int32(efd.hostfd))
+ }
+}
diff --git a/pkg/sentry/fsimpl/eventfd/eventfd_test.go b/pkg/sentry/fsimpl/eventfd/eventfd_test.go
new file mode 100644
index 000000000..20e3adffc
--- /dev/null
+++ b/pkg/sentry/fsimpl/eventfd/eventfd_test.go
@@ -0,0 +1,97 @@
+// 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 eventfd
+
+import (
+ "testing"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/contexttest"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+func TestEventFD(t *testing.T) {
+ initVals := []uint64{
+ 0,
+ // Using a non-zero initial value verifies that writing to an
+ // eventfd signals when the eventfd's counter was already
+ // non-zero.
+ 343,
+ }
+
+ for _, initVal := range initVals {
+ ctx := contexttest.Context(t)
+ vfsObj := &vfs.VirtualFilesystem{}
+ if err := vfsObj.Init(); err != nil {
+ t.Fatalf("VFS init: %v", err)
+ }
+
+ // Make a new eventfd that is writable.
+ eventfd, err := New(vfsObj, initVal, false, linux.O_RDWR)
+ if err != nil {
+ t.Fatalf("New() failed: %v", err)
+ }
+ defer eventfd.DecRef()
+
+ // Register a callback for a write event.
+ w, ch := waiter.NewChannelEntry(nil)
+ eventfd.EventRegister(&w, waiter.EventIn)
+ defer eventfd.EventUnregister(&w)
+
+ data := []byte("00000124")
+ // Create and submit a write request.
+ n, err := eventfd.Write(ctx, usermem.BytesIOSequence(data), vfs.WriteOptions{})
+ if err != nil {
+ t.Fatal(err)
+ }
+ if n != 8 {
+ t.Errorf("eventfd.write wrote %d bytes, not full int64", n)
+ }
+
+ // Check if the callback fired due to the write event.
+ select {
+ case <-ch:
+ default:
+ t.Errorf("Didn't get notified of EventIn after write")
+ }
+ }
+}
+
+func TestEventFDStat(t *testing.T) {
+ ctx := contexttest.Context(t)
+ vfsObj := &vfs.VirtualFilesystem{}
+ if err := vfsObj.Init(); err != nil {
+ t.Fatalf("VFS init: %v", err)
+ }
+
+ // Make a new eventfd that is writable.
+ eventfd, err := New(vfsObj, 0, false, linux.O_RDWR)
+ if err != nil {
+ t.Fatalf("New() failed: %v", err)
+ }
+ defer eventfd.DecRef()
+
+ statx, err := eventfd.Stat(ctx, vfs.StatOptions{
+ Mask: linux.STATX_BASIC_STATS,
+ })
+ if err != nil {
+ t.Fatalf("eventfd.Stat failed: %v", err)
+ }
+ if statx.Size != 0 {
+ t.Errorf("eventfd size should be 0")
+ }
+}
diff --git a/pkg/sentry/fsimpl/ext/ext.go b/pkg/sentry/fsimpl/ext/ext.go
index 7176af6d1..dac6effbf 100644
--- a/pkg/sentry/fsimpl/ext/ext.go
+++ b/pkg/sentry/fsimpl/ext/ext.go
@@ -105,36 +105,50 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
// EACCESS should be returned according to mount(2). Filesystem independent
// flags (like readonly) are currently not available in pkg/sentry/vfs.
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+
dev, err := getDeviceFd(source, opts)
if err != nil {
return nil, nil, err
}
- fs := filesystem{dev: dev, inodeCache: make(map[uint32]*inode)}
+ fs := filesystem{
+ dev: dev,
+ inodeCache: make(map[uint32]*inode),
+ devMinor: devMinor,
+ }
fs.vfsfs.Init(vfsObj, &fsType, &fs)
fs.sb, err = readSuperBlock(dev)
if err != nil {
+ fs.vfsfs.DecRef()
return nil, nil, err
}
if fs.sb.Magic() != linux.EXT_SUPER_MAGIC {
// mount(2) specifies that EINVAL should be returned if the superblock is
// invalid.
+ fs.vfsfs.DecRef()
return nil, nil, syserror.EINVAL
}
// Refuse to mount if the filesystem is incompatible.
if !isCompatible(fs.sb) {
+ fs.vfsfs.DecRef()
return nil, nil, syserror.EINVAL
}
fs.bgs, err = readBlockGroups(dev, fs.sb)
if err != nil {
+ fs.vfsfs.DecRef()
return nil, nil, err
}
rootInode, err := fs.getOrCreateInodeLocked(disklayout.RootDirInode)
if err != nil {
+ fs.vfsfs.DecRef()
return nil, nil, err
}
rootInode.incRef()
diff --git a/pkg/sentry/fsimpl/ext/filesystem.go b/pkg/sentry/fsimpl/ext/filesystem.go
index 77b644275..557963e03 100644
--- a/pkg/sentry/fsimpl/ext/filesystem.go
+++ b/pkg/sentry/fsimpl/ext/filesystem.go
@@ -64,6 +64,10 @@ type filesystem struct {
// bgs represents all the block group descriptors for the filesystem.
// Immutable after initialization.
bgs []disklayout.BlockGroup
+
+ // devMinor is this filesystem's device minor number. Immutable after
+ // initialization.
+ devMinor uint32
}
// Compiles only if filesystem implements vfs.FilesystemImpl.
@@ -366,7 +370,9 @@ func (fs *filesystem) StatFSAt(ctx context.Context, rp *vfs.ResolvingPath) (linu
}
// Release implements vfs.FilesystemImpl.Release.
-func (fs *filesystem) Release() {}
+func (fs *filesystem) Release() {
+ fs.vfsfs.VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+}
// Sync implements vfs.FilesystemImpl.Sync.
func (fs *filesystem) Sync(ctx context.Context) error {
diff --git a/pkg/sentry/fsimpl/ext/inode.go b/pkg/sentry/fsimpl/ext/inode.go
index a98512350..485f86f4b 100644
--- a/pkg/sentry/fsimpl/ext/inode.go
+++ b/pkg/sentry/fsimpl/ext/inode.go
@@ -204,6 +204,8 @@ func (in *inode) statTo(stat *linux.Statx) {
stat.Atime = in.diskInode.AccessTime().StatxTimestamp()
stat.Ctime = in.diskInode.ChangeTime().StatxTimestamp()
stat.Mtime = in.diskInode.ModificationTime().StatxTimestamp()
+ stat.DevMajor = linux.UNNAMED_MAJOR
+ stat.DevMinor = in.fs.devMinor
// TODO(b/134676337): Set stat.Blocks which is the number of 512 byte blocks
// (including metadata blocks) required to represent this file.
}
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 4a32821bd..7f2181216 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -21,6 +21,8 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/p9"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
@@ -835,6 +837,9 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
if d.isSynthetic() {
return nil, syserror.ENXIO
}
+ if d.fs.iopts.OpenSocketsByConnecting {
+ return d.connectSocketLocked(ctx, opts)
+ }
case linux.S_IFIFO:
if d.isSynthetic() {
return d.pipe.Open(ctx, mnt, &d.vfsd, opts.Flags)
@@ -843,10 +848,28 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return d.openSpecialFileLocked(ctx, mnt, opts)
}
+func (d *dentry) connectSocketLocked(ctx context.Context, opts *vfs.OpenOptions) (*vfs.FileDescription, error) {
+ if opts.Flags&linux.O_DIRECT != 0 {
+ return nil, syserror.EINVAL
+ }
+ fdObj, err := d.file.connect(ctx, p9.AnonymousSocket)
+ if err != nil {
+ return nil, err
+ }
+ fd, err := host.NewFD(ctx, kernel.KernelFromContext(ctx).HostMount(), fdObj.FD(), &host.NewFDOptions{
+ HaveFlags: true,
+ Flags: opts.Flags,
+ })
+ if err != nil {
+ fdObj.Close()
+ return nil, err
+ }
+ fdObj.Release()
+ return fd, nil
+}
+
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
}
@@ -854,10 +877,15 @@ func (d *dentry) openSpecialFileLocked(ctx context.Context, mnt *vfs.Mount, opts
if err != nil {
return nil, err
}
+ seekable := d.fileType() == linux.S_IFREG
fd := &specialFileFD{
- handle: h,
+ handle: h,
+ seekable: seekable,
}
- if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{}); err != nil {
+ if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{
+ DenyPRead: !seekable,
+ DenyPWrite: !seekable,
+ }); err != nil {
h.close(ctx)
return nil, err
}
@@ -888,7 +916,11 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
}
creds := rp.Credentials()
name := rp.Component()
- fdobj, openFile, createQID, _, err := dirfile.create(ctx, name, (p9.OpenFlags)(opts.Flags), (p9.FileMode)(opts.Mode), (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
+ // Filter file creation flags and O_LARGEFILE out; the create RPC already
+ // has the semantics of O_CREAT|O_EXCL, while some servers will choke on
+ // O_LARGEFILE.
+ createFlags := p9.OpenFlags(opts.Flags &^ (linux.O_CREAT | linux.O_EXCL | linux.O_NOCTTY | linux.O_TRUNC | linux.O_LARGEFILE))
+ fdobj, openFile, createQID, _, err := dirfile.create(ctx, name, createFlags, (p9.FileMode)(opts.Mode), (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
if err != nil {
dirfile.close(ctx)
return nil, err
@@ -896,7 +928,7 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
// Then we need to walk to the file we just created to get a non-open fid
// representing it, and to get its metadata. This must use d.file since, as
// explained above, dirfile was invalidated by dirfile.Create().
- walkQID, nonOpenFile, attrMask, attr, err := d.file.walkGetAttrOne(ctx, name)
+ _, nonOpenFile, attrMask, attr, err := d.file.walkGetAttrOne(ctx, name)
if err != nil {
openFile.close(ctx)
if fdobj != nil {
@@ -904,17 +936,6 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
}
return nil, err
}
- // Sanity-check that we walked to the file we created.
- if createQID.Path != walkQID.Path {
- // Probably due to concurrent remote filesystem mutation?
- ctx.Warningf("gofer.dentry.createAndOpenChildLocked: created file has QID %v before walk, QID %v after (interop=%v)", createQID, walkQID, d.fs.opts.interop)
- nonOpenFile.close(ctx)
- openFile.close(ctx)
- if fdobj != nil {
- fdobj.Close()
- }
- return nil, syserror.EAGAIN
- }
// Construct the new dentry.
child, err := d.fs.newDentry(ctx, nonOpenFile, createQID, attrMask, &attr)
@@ -960,16 +981,21 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
}
childVFSFD = &fd.vfsfd
} else {
+ seekable := child.fileType() == linux.S_IFREG
fd := &specialFileFD{
handle: handle{
file: openFile,
fd: -1,
},
+ seekable: seekable,
}
if fdobj != nil {
fd.handle.fd = int32(fdobj.Release())
}
- if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &child.vfsd, &vfs.FileDescriptionOptions{}); err != nil {
+ if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &child.vfsd, &vfs.FileDescriptionOptions{
+ DenyPRead: !seekable,
+ DenyPWrite: !seekable,
+ }); err != nil {
fd.handle.close(ctx)
return nil, err
}
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index 9ab8fdc65..1da8d5d82 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -81,6 +81,9 @@ type filesystem struct {
// clock is a realtime clock used to set timestamps in file operations.
clock ktime.Clock
+ // devMinor is the filesystem's minor device number. devMinor is immutable.
+ devMinor uint32
+
// uid and gid are the effective KUID and KGID of the filesystem's creator,
// and are used as the owner and group for files that don't specify one.
// uid and gid are immutable.
@@ -218,6 +221,10 @@ type InternalFilesystemOptions struct {
// which servers can handle only a single client and report failure if that
// client disconnects.
LeakConnection bool
+
+ // If OpenSocketsByConnecting is true, silently translate attempts to open
+ // files identifying as sockets to connect RPCs.
+ OpenSocketsByConnecting bool
}
// Name implements vfs.FilesystemType.Name.
@@ -399,14 +406,21 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
}
// Construct the filesystem object.
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ attachFile.close(ctx)
+ client.Close()
+ return nil, nil, err
+ }
fs := &filesystem{
mfp: mfp,
opts: fsopts,
iopts: iopts,
- uid: creds.EffectiveKUID,
- gid: creds.EffectiveKGID,
client: client,
clock: ktime.RealtimeClockFromContext(ctx),
+ devMinor: devMinor,
+ uid: creds.EffectiveKUID,
+ gid: creds.EffectiveKGID,
syncableDentries: make(map[*dentry]struct{}),
specialFileFDs: make(map[*specialFileFD]struct{}),
}
@@ -464,6 +478,8 @@ func (fs *filesystem) Release() {
// Close the connection to the server. This implicitly clunks all fids.
fs.client.Close()
}
+
+ fs.vfsfs.VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
}
// dentry implements vfs.DentryImpl.
@@ -796,7 +812,8 @@ func (d *dentry) statTo(stat *linux.Statx) {
stat.Btime = statxTimestampFromDentry(atomic.LoadInt64(&d.btime))
stat.Ctime = statxTimestampFromDentry(atomic.LoadInt64(&d.ctime))
stat.Mtime = statxTimestampFromDentry(atomic.LoadInt64(&d.mtime))
- // TODO(gvisor.dev/issue/1198): device number
+ stat.DevMajor = linux.UNNAMED_MAJOR
+ stat.DevMinor = d.fs.devMinor
}
func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *linux.Statx, mnt *vfs.Mount) error {
diff --git a/pkg/sentry/fsimpl/gofer/special_file.go b/pkg/sentry/fsimpl/gofer/special_file.go
index 507e0e276..a464e6a94 100644
--- a/pkg/sentry/fsimpl/gofer/special_file.go
+++ b/pkg/sentry/fsimpl/gofer/special_file.go
@@ -33,13 +33,14 @@ import (
type specialFileFD struct {
fileDescription
- // handle is immutable.
+ // handle is used for file I/O. handle is immutable.
handle handle
- // off is the file offset. off is protected by mu. (POSIX 2.9.7 only
- // requires operations using the file offset to be atomic for regular files
- // and symlinks; however, since specialFileFD may be used for regular
- // files, we apply this atomicity unconditionally.)
+ // seekable is true if this file description represents a file for which
+ // file offset is significant, i.e. a regular file. seekable is immutable.
+ seekable bool
+
+ // If seekable is true, off is the file offset. off is protected by mu.
mu sync.Mutex
off int64
}
@@ -63,7 +64,7 @@ func (fd *specialFileFD) OnClose(ctx context.Context) error {
// PRead implements vfs.FileDescriptionImpl.PRead.
func (fd *specialFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
- if offset < 0 {
+ if fd.seekable && offset < 0 {
return 0, syserror.EINVAL
}
if opts.Flags != 0 {
@@ -91,6 +92,10 @@ func (fd *specialFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs
// Read implements vfs.FileDescriptionImpl.Read.
func (fd *specialFileFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+ if !fd.seekable {
+ return fd.PRead(ctx, dst, -1, opts)
+ }
+
fd.mu.Lock()
n, err := fd.PRead(ctx, dst, fd.off, opts)
fd.off += n
@@ -100,14 +105,14 @@ func (fd *specialFileFD) Read(ctx context.Context, dst usermem.IOSequence, opts
// PWrite implements vfs.FileDescriptionImpl.PWrite.
func (fd *specialFileFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
- if offset < 0 {
+ if fd.seekable && offset < 0 {
return 0, syserror.EINVAL
}
if opts.Flags != 0 {
return 0, syserror.EOPNOTSUPP
}
- if fd.dentry().fileType() == linux.S_IFREG {
+ if fd.seekable {
limit, err := vfs.CheckLimit(ctx, offset, src.NumBytes())
if err != nil {
return 0, err
@@ -130,6 +135,10 @@ func (fd *specialFileFD) PWrite(ctx context.Context, src usermem.IOSequence, off
// Write implements vfs.FileDescriptionImpl.Write.
func (fd *specialFileFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+ if !fd.seekable {
+ return fd.PWrite(ctx, src, -1, opts)
+ }
+
fd.mu.Lock()
n, err := fd.PWrite(ctx, src, fd.off, opts)
fd.off += n
@@ -139,6 +148,9 @@ func (fd *specialFileFD) Write(ctx context.Context, src usermem.IOSequence, opts
// Seek implements vfs.FileDescriptionImpl.Seek.
func (fd *specialFileFD) Seek(ctx context.Context, offset int64, whence int32) (int64, error) {
+ if !fd.seekable {
+ return 0, syserror.ESPIPE
+ }
fd.mu.Lock()
defer fd.mu.Unlock()
switch whence {
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index 144e04905..8caf55a1b 100644
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -40,8 +40,20 @@ import (
"gvisor.dev/gvisor/pkg/waiter"
)
-// 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) {
+// NewFDOptions contains options to NewFD.
+type NewFDOptions struct {
+ // If IsTTY is true, the file descriptor is a TTY.
+ IsTTY bool
+
+ // If HaveFlags is true, use Flags for the new file description. Otherwise,
+ // the new file description will inherit flags from hostFD.
+ HaveFlags bool
+ Flags uint32
+}
+
+// NewFD returns a vfs.FileDescription representing the given host file
+// descriptor. mnt must be Kernel.HostMount().
+func NewFD(ctx context.Context, mnt *vfs.Mount, hostFD int, opts *NewFDOptions) (*vfs.FileDescription, error) {
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())
@@ -53,6 +65,16 @@ func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs
return nil, err
}
+ flags := opts.Flags
+ if !opts.HaveFlags {
+ // Get flags for the imported FD.
+ flagsInt, err := unix.FcntlInt(uintptr(hostFD), syscall.F_GETFL, 0)
+ if err != nil {
+ return nil, err
+ }
+ flags = uint32(flagsInt)
+ }
+
fileMode := linux.FileMode(s.Mode)
fileType := fileMode.FileType()
@@ -65,7 +87,7 @@ func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs
i := &inode{
hostFD: hostFD,
seekable: seekable,
- isTTY: isTTY,
+ isTTY: opts.IsTTY,
canMap: canMap(uint32(fileType)),
wouldBlock: wouldBlock(uint32(fileType)),
ino: fs.NextIno(),
@@ -95,7 +117,14 @@ func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs
// i.open will take a reference on d.
defer d.DecRef()
- return i.open(ctx, d.VFSDentry(), mnt)
+ return i.open(ctx, d.VFSDentry(), mnt, flags)
+}
+
+// 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) {
+ return NewFD(ctx, mnt, hostFD, &NewFDOptions{
+ IsTTY: isTTY,
+ })
}
// filesystemType implements vfs.FilesystemType.
@@ -115,15 +144,28 @@ func (filesystemType) Name() string {
//
// 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{}
+func NewFilesystem(vfsObj *vfs.VirtualFilesystem) (*vfs.Filesystem, error) {
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, err
+ }
+ fs := &filesystem{
+ devMinor: devMinor,
+ }
fs.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
- return fs.VFSFilesystem()
+ return fs.VFSFilesystem(), nil
}
// filesystem implements vfs.FilesystemImpl.
type filesystem struct {
kernfs.Filesystem
+
+ devMinor uint32
+}
+
+func (fs *filesystem) Release() {
+ fs.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release()
}
func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDentry, b *fspath.Builder) error {
@@ -185,19 +227,6 @@ type inode struct {
queue waiter.Queue
}
-// Note that these flags may become out of date, since they can be modified
-// on the host, e.g. with fcntl.
-func fileFlagsFromHostFD(fd int) (uint32, error) {
- flags, err := unix.FcntlInt(uintptr(fd), syscall.F_GETFL, 0)
- if err != nil {
- log.Warningf("Failed to get file flags for donated FD %d: %v", fd, err)
- return 0, err
- }
- // TODO(gvisor.dev/issue/1672): implement behavior corresponding to these allowed flags.
- flags &= syscall.O_ACCMODE | syscall.O_DIRECT | syscall.O_NONBLOCK | syscall.O_DSYNC | syscall.O_SYNC | syscall.O_APPEND
- return uint32(flags), nil
-}
-
// CheckPermissions implements kernfs.Inode.
func (i *inode) CheckPermissions(ctx context.Context, creds *auth.Credentials, ats vfs.AccessTypes) error {
var s syscall.Stat_t
@@ -219,7 +248,7 @@ func (i *inode) Mode() linux.FileMode {
}
// Stat implements kernfs.Inode.
-func (i *inode) Stat(_ *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) {
+func (i *inode) Stat(vfsfs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) {
if opts.Mask&linux.STATX__RESERVED != 0 {
return linux.Statx{}, syserror.EINVAL
}
@@ -227,73 +256,73 @@ func (i *inode) Stat(_ *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, erro
return linux.Statx{}, syserror.EINVAL
}
+ fs := vfsfs.Impl().(*filesystem)
+
// Limit our host call only to known flags.
mask := opts.Mask & linux.STATX_ALL
var s unix.Statx_t
err := unix.Statx(i.hostFD, "", int(unix.AT_EMPTY_PATH|opts.Sync), int(mask), &s)
- // Fallback to fstat(2), if statx(2) is not supported on the host.
- //
- // TODO(b/151263641): Remove fallback.
if err == syserror.ENOSYS {
- return i.fstat(opts)
- } else if err != nil {
+ // Fallback to fstat(2), if statx(2) is not supported on the host.
+ //
+ // TODO(b/151263641): Remove fallback.
+ return i.fstat(fs)
+ }
+ if err != nil {
return linux.Statx{}, err
}
- ls := linux.Statx{Mask: mask}
- // Unconditionally fill blksize, attributes, and device numbers, as indicated
- // by /include/uapi/linux/stat.h.
- //
- // RdevMajor/RdevMinor are left as zero, so as not to expose host device
- // numbers.
- //
- // TODO(gvisor.dev/issue/1672): Use kernfs-specific, internally defined
- // device numbers. If we use the device number from the host, it may collide
- // with another sentry-internal device number. We handle device/inode
- // numbers without relying on the host to prevent collisions.
- ls.Blksize = s.Blksize
- ls.Attributes = s.Attributes
- ls.AttributesMask = s.Attributes_mask
-
- if mask&linux.STATX_TYPE != 0 {
+ // Unconditionally fill blksize, attributes, and device numbers, as
+ // indicated by /include/uapi/linux/stat.h. Inode number is always
+ // available, since we use our own rather than the host's.
+ ls := linux.Statx{
+ Mask: linux.STATX_INO,
+ Blksize: s.Blksize,
+ Attributes: s.Attributes,
+ Ino: i.ino,
+ AttributesMask: s.Attributes_mask,
+ DevMajor: linux.UNNAMED_MAJOR,
+ DevMinor: fs.devMinor,
+ }
+
+ // Copy other fields that were returned by the host. RdevMajor/RdevMinor
+ // are never copied (and therefore left as zero), so as not to expose host
+ // device numbers.
+ ls.Mask |= s.Mask & linux.STATX_ALL
+ if s.Mask&linux.STATX_TYPE != 0 {
ls.Mode |= s.Mode & linux.S_IFMT
}
- if mask&linux.STATX_MODE != 0 {
+ if s.Mask&linux.STATX_MODE != 0 {
ls.Mode |= s.Mode &^ linux.S_IFMT
}
- if mask&linux.STATX_NLINK != 0 {
+ if s.Mask&linux.STATX_NLINK != 0 {
ls.Nlink = s.Nlink
}
- if mask&linux.STATX_UID != 0 {
+ if s.Mask&linux.STATX_UID != 0 {
ls.UID = s.Uid
}
- if mask&linux.STATX_GID != 0 {
+ if s.Mask&linux.STATX_GID != 0 {
ls.GID = s.Gid
}
- if mask&linux.STATX_ATIME != 0 {
+ if s.Mask&linux.STATX_ATIME != 0 {
ls.Atime = unixToLinuxStatxTimestamp(s.Atime)
}
- if mask&linux.STATX_BTIME != 0 {
+ if s.Mask&linux.STATX_BTIME != 0 {
ls.Btime = unixToLinuxStatxTimestamp(s.Btime)
}
- if mask&linux.STATX_CTIME != 0 {
+ if s.Mask&linux.STATX_CTIME != 0 {
ls.Ctime = unixToLinuxStatxTimestamp(s.Ctime)
}
- if mask&linux.STATX_MTIME != 0 {
+ if s.Mask&linux.STATX_MTIME != 0 {
ls.Mtime = unixToLinuxStatxTimestamp(s.Mtime)
}
- if mask&linux.STATX_SIZE != 0 {
+ if s.Mask&linux.STATX_SIZE != 0 {
ls.Size = s.Size
}
- if mask&linux.STATX_BLOCKS != 0 {
+ if s.Mask&linux.STATX_BLOCKS != 0 {
ls.Blocks = s.Blocks
}
- // Use our own internal inode number.
- if mask&linux.STATX_INO != 0 {
- ls.Ino = i.ino
- }
-
return ls, nil
}
@@ -305,36 +334,30 @@ func (i *inode) Stat(_ *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, erro
// of a mask or sync flags. fstat(2) does not provide any metadata
// equivalent to Statx.Attributes, Statx.AttributesMask, or Statx.Btime, so
// those fields remain empty.
-func (i *inode) fstat(opts vfs.StatOptions) (linux.Statx, error) {
+func (i *inode) fstat(fs *filesystem) (linux.Statx, error) {
var s unix.Stat_t
if err := unix.Fstat(i.hostFD, &s); err != nil {
return linux.Statx{}, err
}
- // Note that rdev numbers are left as 0; do not expose host device numbers.
- ls := linux.Statx{
- Mask: linux.STATX_BASIC_STATS,
- Blksize: uint32(s.Blksize),
- Nlink: uint32(s.Nlink),
- UID: s.Uid,
- GID: s.Gid,
- Mode: uint16(s.Mode),
- Size: uint64(s.Size),
- Blocks: uint64(s.Blocks),
- Atime: timespecToStatxTimestamp(s.Atim),
- Ctime: timespecToStatxTimestamp(s.Ctim),
- Mtime: timespecToStatxTimestamp(s.Mtim),
- }
-
- // Use our own internal inode number.
- //
- // TODO(gvisor.dev/issue/1672): Use a kernfs-specific device number as well.
- // If we use the device number from the host, it may collide with another
- // sentry-internal device number. We handle device/inode numbers without
- // relying on the host to prevent collisions.
- ls.Ino = i.ino
-
- return ls, nil
+ // As with inode.Stat(), we always use internal device and inode numbers,
+ // and never expose the host's represented device numbers.
+ return linux.Statx{
+ Mask: linux.STATX_BASIC_STATS,
+ Blksize: uint32(s.Blksize),
+ Nlink: uint32(s.Nlink),
+ UID: s.Uid,
+ GID: s.Gid,
+ Mode: uint16(s.Mode),
+ Ino: i.ino,
+ Size: uint64(s.Size),
+ Blocks: uint64(s.Blocks),
+ Atime: timespecToStatxTimestamp(s.Atim),
+ Ctime: timespecToStatxTimestamp(s.Ctim),
+ Mtime: timespecToStatxTimestamp(s.Mtim),
+ DevMajor: linux.UNNAMED_MAJOR,
+ DevMinor: fs.devMinor,
+ }, nil
}
// SetStat implements kernfs.Inode.
@@ -399,19 +422,19 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr
if i.Mode().FileType() == linux.S_IFSOCK {
return nil, syserror.ENXIO
}
- return i.open(ctx, vfsd, rp.Mount())
+ return i.open(ctx, vfsd, rp.Mount(), opts.Flags)
}
-func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount) (*vfs.FileDescription, error) {
+func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount, flags uint32) (*vfs.FileDescription, error) {
var s syscall.Stat_t
if err := syscall.Fstat(i.hostFD, &s); err != nil {
return nil, err
}
fileType := s.Mode & linux.FileTypeMask
- flags, err := fileFlagsFromHostFD(i.hostFD)
- if err != nil {
- return nil, err
- }
+
+ // Constrain flags to a subset we can handle.
+ // TODO(gvisor.dev/issue/1672): implement behavior corresponding to these allowed flags.
+ flags &= syscall.O_ACCMODE | syscall.O_DIRECT | syscall.O_NONBLOCK | syscall.O_DSYNC | syscall.O_SYNC | syscall.O_APPEND
if fileType == syscall.S_IFSOCK {
if i.isTTY {
@@ -453,8 +476,6 @@ func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount) (*vfs.F
}
// fileDescription is embedded by host fd implementations of FileDescriptionImpl.
-//
-// TODO(gvisor.dev/issue/1672): Implement Waitable interface.
type fileDescription struct {
vfsfd vfs.FileDescription
vfs.FileDescriptionDefaultImpl
@@ -471,12 +492,12 @@ type fileDescription struct {
// SetStat implements vfs.FileDescriptionImpl.
func (f *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error {
creds := auth.CredentialsFromContext(ctx)
- return f.inode.SetStat(ctx, nil, creds, opts)
+ return f.inode.SetStat(ctx, f.vfsfd.Mount().Filesystem(), creds, opts)
}
// Stat implements vfs.FileDescriptionImpl.
func (f *fileDescription) Stat(_ context.Context, opts vfs.StatOptions) (linux.Statx, error) {
- return f.inode.Stat(nil, opts)
+ return f.inode.Stat(f.vfsfd.Mount().Filesystem(), opts)
}
// Release implements vfs.FileDescriptionImpl.
diff --git a/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go b/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
index c7779fc11..1568a9d49 100644
--- a/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
+++ b/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
@@ -44,11 +44,11 @@ type DynamicBytesFile struct {
var _ Inode = (*DynamicBytesFile)(nil)
// Init initializes a dynamic bytes file.
-func (f *DynamicBytesFile) Init(creds *auth.Credentials, ino uint64, data vfs.DynamicBytesSource, perm linux.FileMode) {
+func (f *DynamicBytesFile) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, data vfs.DynamicBytesSource, perm linux.FileMode) {
if perm&^linux.PermissionsMask != 0 {
panic(fmt.Sprintf("Only permission mask must be set: %x", perm&linux.PermissionsMask))
}
- f.InodeAttrs.Init(creds, ino, linux.ModeRegular|perm)
+ f.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.ModeRegular|perm)
f.data = data
}
diff --git a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
index 615592d5f..982daa2e6 100644
--- a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
+++ b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
@@ -192,15 +192,17 @@ func (InodeNotSymlink) Getlink(context.Context, *vfs.Mount) (vfs.VirtualDentry,
//
// Must be initialized by Init prior to first use.
type InodeAttrs struct {
- ino uint64
- mode uint32
- uid uint32
- gid uint32
- nlink uint32
+ devMajor uint32
+ devMinor uint32
+ ino uint64
+ mode uint32
+ uid uint32
+ gid uint32
+ nlink uint32
}
// Init initializes this InodeAttrs.
-func (a *InodeAttrs) Init(creds *auth.Credentials, ino uint64, mode linux.FileMode) {
+func (a *InodeAttrs) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, mode linux.FileMode) {
if mode.FileType() == 0 {
panic(fmt.Sprintf("No file type specified in 'mode' for InodeAttrs.Init(): mode=0%o", mode))
}
@@ -209,6 +211,8 @@ func (a *InodeAttrs) Init(creds *auth.Credentials, ino uint64, mode linux.FileMo
if mode.FileType() == linux.ModeDirectory {
nlink = 2
}
+ a.devMajor = devMajor
+ a.devMinor = devMinor
atomic.StoreUint64(&a.ino, ino)
atomic.StoreUint32(&a.mode, uint32(mode))
atomic.StoreUint32(&a.uid, uint32(creds.EffectiveKUID))
@@ -216,6 +220,16 @@ func (a *InodeAttrs) Init(creds *auth.Credentials, ino uint64, mode linux.FileMo
atomic.StoreUint32(&a.nlink, nlink)
}
+// DevMajor returns the device major number.
+func (a *InodeAttrs) DevMajor() uint32 {
+ return a.devMajor
+}
+
+// DevMinor returns the device minor number.
+func (a *InodeAttrs) DevMinor() uint32 {
+ return a.devMinor
+}
+
// Ino returns the inode id.
func (a *InodeAttrs) Ino() uint64 {
return atomic.LoadUint64(&a.ino)
@@ -232,6 +246,8 @@ func (a *InodeAttrs) Mode() linux.FileMode {
func (a *InodeAttrs) Stat(*vfs.Filesystem, vfs.StatOptions) (linux.Statx, error) {
var stat linux.Statx
stat.Mask = linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID | linux.STATX_INO | linux.STATX_NLINK
+ stat.DevMajor = a.devMajor
+ stat.DevMinor = a.devMinor
stat.Ino = atomic.LoadUint64(&a.ino)
stat.Mode = uint16(a.Mode())
stat.UID = atomic.LoadUint32(&a.uid)
@@ -544,9 +560,9 @@ type StaticDirectory struct {
var _ Inode = (*StaticDirectory)(nil)
// NewStaticDir creates a new static directory and returns its dentry.
-func NewStaticDir(creds *auth.Credentials, ino uint64, perm linux.FileMode, children map[string]*Dentry) *Dentry {
+func NewStaticDir(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode, children map[string]*Dentry) *Dentry {
inode := &StaticDirectory{}
- inode.Init(creds, ino, perm)
+ inode.Init(creds, devMajor, devMinor, ino, perm)
dentry := &Dentry{}
dentry.Init(inode)
@@ -559,11 +575,11 @@ func NewStaticDir(creds *auth.Credentials, ino uint64, perm linux.FileMode, chil
}
// Init initializes StaticDirectory.
-func (s *StaticDirectory) Init(creds *auth.Credentials, ino uint64, perm linux.FileMode) {
+func (s *StaticDirectory) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode) {
if perm&^linux.PermissionsMask != 0 {
panic(fmt.Sprintf("Only permission mask must be set: %x", perm&linux.PermissionsMask))
}
- s.InodeAttrs.Init(creds, ino, linux.ModeDirectory|perm)
+ s.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.ModeDirectory|perm)
}
// Open implements kernfs.Inode.
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs_test.go b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
index 1c5d3e7e7..412cf6ac9 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs_test.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
@@ -75,7 +75,7 @@ type file struct {
func (fs *filesystem) newFile(creds *auth.Credentials, content string) *kernfs.Dentry {
f := &file{}
f.content = content
- f.DynamicBytesFile.Init(creds, fs.NextIno(), f, 0777)
+ f.DynamicBytesFile.Init(creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), f, 0777)
d := &kernfs.Dentry{}
d.Init(f)
@@ -107,7 +107,7 @@ type readonlyDir struct {
func (fs *filesystem) newReadonlyDir(creds *auth.Credentials, mode linux.FileMode, contents map[string]*kernfs.Dentry) *kernfs.Dentry {
dir := &readonlyDir{}
- dir.attrs.Init(creds, fs.NextIno(), linux.ModeDirectory|mode)
+ dir.attrs.Init(creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), linux.ModeDirectory|mode)
dir.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
dir.dentry.Init(dir)
@@ -137,7 +137,7 @@ type dir struct {
func (fs *filesystem) newDir(creds *auth.Credentials, mode linux.FileMode, contents map[string]*kernfs.Dentry) *kernfs.Dentry {
dir := &dir{}
dir.fs = fs
- dir.attrs.Init(creds, fs.NextIno(), linux.ModeDirectory|mode)
+ dir.attrs.Init(creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), linux.ModeDirectory|mode)
dir.OrderedChildren.Init(kernfs.OrderedChildrenOptions{Writable: true})
dir.dentry.Init(dir)
diff --git a/pkg/sentry/fsimpl/kernfs/symlink.go b/pkg/sentry/fsimpl/kernfs/symlink.go
index 0aa6dc979..2ab3f53fd 100644
--- a/pkg/sentry/fsimpl/kernfs/symlink.go
+++ b/pkg/sentry/fsimpl/kernfs/symlink.go
@@ -35,9 +35,9 @@ type StaticSymlink struct {
var _ Inode = (*StaticSymlink)(nil)
// NewStaticSymlink creates a new symlink file pointing to 'target'.
-func NewStaticSymlink(creds *auth.Credentials, ino uint64, target string) *Dentry {
+func NewStaticSymlink(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, target string) *Dentry {
inode := &StaticSymlink{}
- inode.Init(creds, ino, target)
+ inode.Init(creds, devMajor, devMinor, ino, target)
d := &Dentry{}
d.Init(inode)
@@ -45,9 +45,9 @@ func NewStaticSymlink(creds *auth.Credentials, ino uint64, target string) *Dentr
}
// Init initializes the instance.
-func (s *StaticSymlink) Init(creds *auth.Credentials, ino uint64, target string) {
+func (s *StaticSymlink) Init(creds *auth.Credentials, devMajor uint32, devMinor uint32, ino uint64, target string) {
s.target = target
- s.InodeAttrs.Init(creds, ino, linux.ModeSymlink|0777)
+ s.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.ModeSymlink|0777)
}
// Readlink implements Inode.
diff --git a/pkg/sentry/fsimpl/pipefs/BUILD b/pkg/sentry/fsimpl/pipefs/BUILD
index 0d411606f..5950a2d59 100644
--- a/pkg/sentry/fsimpl/pipefs/BUILD
+++ b/pkg/sentry/fsimpl/pipefs/BUILD
@@ -9,6 +9,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/fspath",
"//pkg/sentry/fsimpl/kernfs",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/pipe",
diff --git a/pkg/sentry/fsimpl/pipefs/pipefs.go b/pkg/sentry/fsimpl/pipefs/pipefs.go
index 5375e5e75..cab771211 100644
--- a/pkg/sentry/fsimpl/pipefs/pipefs.go
+++ b/pkg/sentry/fsimpl/pipefs/pipefs.go
@@ -17,8 +17,11 @@
package pipefs
import (
+ "fmt"
+
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
@@ -40,20 +43,36 @@ func (filesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFile
panic("pipefs.filesystemType.GetFilesystem should never be called")
}
-// 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).
+type filesystem struct {
+ kernfs.Filesystem
+
+ devMinor uint32
+}
// NewFilesystem sets up and returns a new vfs.Filesystem implemented by pipefs.
-func NewFilesystem(vfsObj *vfs.VirtualFilesystem) *vfs.Filesystem {
- fs := &kernfs.Filesystem{}
- fs.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
- return fs.VFSFilesystem()
+func NewFilesystem(vfsObj *vfs.VirtualFilesystem) (*vfs.Filesystem, error) {
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, err
+ }
+ fs := &filesystem{
+ devMinor: devMinor,
+ }
+ fs.Filesystem.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
+ return fs.Filesystem.VFSFilesystem(), nil
+}
+
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release() {
+ fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release()
+}
+
+// PrependPath implements vfs.FilesystemImpl.PrependPath.
+func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDentry, b *fspath.Builder) error {
+ inode := vd.Dentry().Impl().(*kernfs.Dentry).Inode().(*inode)
+ b.PrependComponent(fmt.Sprintf("pipe:[%d]", inode.ino))
+ return vfs.PrependPathSyntheticError{}
}
// inode implements kernfs.Inode.
@@ -71,11 +90,11 @@ type inode struct {
ctime ktime.Time
}
-func newInode(ctx context.Context, fs *kernfs.Filesystem) *inode {
+func newInode(ctx context.Context, fs *filesystem) *inode {
creds := auth.CredentialsFromContext(ctx)
return &inode{
pipe: pipe.NewVFSPipe(false /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize),
- ino: fs.NextIno(),
+ ino: fs.Filesystem.NextIno(),
uid: creds.EffectiveKUID,
gid: creds.EffectiveKGID,
ctime: ktime.NowFromContext(ctx),
@@ -98,19 +117,20 @@ func (i *inode) Mode() linux.FileMode {
func (i *inode) Stat(vfsfs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) {
ts := linux.NsecToStatxTimestamp(i.ctime.Nanoseconds())
return linux.Statx{
- Mask: linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_NLINK | linux.STATX_UID | linux.STATX_GID | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME | linux.STATX_INO | linux.STATX_SIZE | linux.STATX_BLOCKS,
- Blksize: usermem.PageSize,
- Nlink: 1,
- UID: uint32(i.uid),
- GID: uint32(i.gid),
- Mode: pipeMode,
- Ino: i.ino,
- Size: 0,
- Blocks: 0,
- Atime: ts,
- Ctime: ts,
- Mtime: ts,
- // TODO(gvisor.dev/issue/1197): Device number.
+ Mask: linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_NLINK | linux.STATX_UID | linux.STATX_GID | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME | linux.STATX_INO | linux.STATX_SIZE | linux.STATX_BLOCKS,
+ Blksize: usermem.PageSize,
+ Nlink: 1,
+ UID: uint32(i.uid),
+ GID: uint32(i.gid),
+ Mode: pipeMode,
+ Ino: i.ino,
+ Size: 0,
+ Blocks: 0,
+ Atime: ts,
+ Ctime: ts,
+ Mtime: ts,
+ DevMajor: linux.UNNAMED_MAJOR,
+ DevMinor: vfsfs.Impl().(*filesystem).devMinor,
}, nil
}
@@ -122,6 +142,9 @@ func (i *inode) SetStat(ctx context.Context, vfsfs *vfs.Filesystem, creds *auth.
return syserror.EPERM
}
+// TODO(gvisor.dev/issue/1193): kernfs does not provide a way to implement
+// statfs, from which we should indicate PIPEFS_MAGIC.
+
// Open implements kernfs.Inode.Open.
func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
return i.pipe.Open(ctx, rp.Mount(), vfsd, opts.Flags)
@@ -132,7 +155,7 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr
//
// Preconditions: mnt.Filesystem() must have been returned by NewFilesystem().
func NewConnectedPipeFDs(ctx context.Context, mnt *vfs.Mount, flags uint32) (*vfs.FileDescription, *vfs.FileDescription) {
- fs := mnt.Filesystem().Impl().(*kernfs.Filesystem)
+ fs := mnt.Filesystem().Impl().(*filesystem)
inode := newInode(ctx, fs)
var d kernfs.Dentry
d.Init(inode)
diff --git a/pkg/sentry/fsimpl/proc/filesystem.go b/pkg/sentry/fsimpl/proc/filesystem.go
index 104fc9030..609210253 100644
--- a/pkg/sentry/fsimpl/proc/filesystem.go
+++ b/pkg/sentry/fsimpl/proc/filesystem.go
@@ -41,6 +41,12 @@ func (FilesystemType) Name() string {
return Name
}
+type filesystem struct {
+ kernfs.Filesystem
+
+ devMinor uint32
+}
+
// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
func (ft FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
k := kernel.KernelFromContext(ctx)
@@ -51,8 +57,13 @@ func (ft FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualF
if pidns == nil {
return nil, nil, fmt.Errorf("procfs requires a PID namespace")
}
-
- procfs := &kernfs.Filesystem{}
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+ procfs := &filesystem{
+ devMinor: devMinor,
+ }
procfs.VFSFilesystem().Init(vfsObj, &ft, procfs)
var cgroups map[string]string
@@ -61,21 +72,27 @@ func (ft FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualF
cgroups = data.Cgroups
}
- _, dentry := newTasksInode(procfs, k, pidns, cgroups)
+ _, dentry := procfs.newTasksInode(k, pidns, cgroups)
return procfs.VFSFilesystem(), dentry.VFSDentry(), nil
}
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release() {
+ fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release()
+}
+
// dynamicInode is an overfitted interface for common Inodes with
// dynamicByteSource types used in procfs.
type dynamicInode interface {
kernfs.Inode
vfs.DynamicBytesSource
- Init(creds *auth.Credentials, ino uint64, data vfs.DynamicBytesSource, perm linux.FileMode)
+ Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, data vfs.DynamicBytesSource, perm linux.FileMode)
}
-func newDentry(creds *auth.Credentials, ino uint64, perm linux.FileMode, inode dynamicInode) *kernfs.Dentry {
- inode.Init(creds, ino, inode, perm)
+func (fs *filesystem) newDentry(creds *auth.Credentials, ino uint64, perm linux.FileMode, inode dynamicInode) *kernfs.Dentry {
+ inode.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, ino, inode, perm)
d := &kernfs.Dentry{}
d.Init(inode)
diff --git a/pkg/sentry/fsimpl/proc/subtasks.go b/pkg/sentry/fsimpl/proc/subtasks.go
index a5cfa8333..36a911db4 100644
--- a/pkg/sentry/fsimpl/proc/subtasks.go
+++ b/pkg/sentry/fsimpl/proc/subtasks.go
@@ -37,23 +37,23 @@ type subtasksInode struct {
kernfs.OrderedChildren
kernfs.AlwaysValid
+ fs *filesystem
task *kernel.Task
pidns *kernel.PIDNamespace
- inoGen InoGenerator
cgroupControllers map[string]string
}
var _ kernfs.Inode = (*subtasksInode)(nil)
-func newSubtasks(task *kernel.Task, pidns *kernel.PIDNamespace, inoGen InoGenerator, cgroupControllers map[string]string) *kernfs.Dentry {
+func (fs *filesystem) newSubtasks(task *kernel.Task, pidns *kernel.PIDNamespace, cgroupControllers map[string]string) *kernfs.Dentry {
subInode := &subtasksInode{
+ fs: fs,
task: task,
pidns: pidns,
- inoGen: inoGen,
cgroupControllers: cgroupControllers,
}
// Note: credentials are overridden by taskOwnedInode.
- subInode.InodeAttrs.Init(task.Credentials(), inoGen.NextIno(), linux.ModeDirectory|0555)
+ subInode.InodeAttrs.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0555)
subInode.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
inode := &taskOwnedInode{Inode: subInode, owner: task}
@@ -78,7 +78,7 @@ func (i *subtasksInode) Lookup(ctx context.Context, name string) (*vfs.Dentry, e
return nil, syserror.ENOENT
}
- subTaskDentry := newTaskInode(i.inoGen, subTask, i.pidns, false, i.cgroupControllers)
+ subTaskDentry := i.fs.newTaskInode(subTask, i.pidns, false, i.cgroupControllers)
return subTaskDentry.VFSDentry(), nil
}
@@ -102,7 +102,7 @@ func (i *subtasksInode) IterDirents(ctx context.Context, cb vfs.IterDirentsCallb
dirent := vfs.Dirent{
Name: strconv.FormatUint(uint64(tid), 10),
Type: linux.DT_DIR,
- Ino: i.inoGen.NextIno(),
+ Ino: i.fs.NextIno(),
NextOff: offset + 1,
}
if err := cb.Handle(dirent); err != nil {
diff --git a/pkg/sentry/fsimpl/proc/task.go b/pkg/sentry/fsimpl/proc/task.go
index 66419d91b..482055db1 100644
--- a/pkg/sentry/fsimpl/proc/task.go
+++ b/pkg/sentry/fsimpl/proc/task.go
@@ -43,45 +43,45 @@ type taskInode struct {
var _ kernfs.Inode = (*taskInode)(nil)
-func newTaskInode(inoGen InoGenerator, task *kernel.Task, pidns *kernel.PIDNamespace, isThreadGroup bool, cgroupControllers map[string]string) *kernfs.Dentry {
+func (fs *filesystem) newTaskInode(task *kernel.Task, pidns *kernel.PIDNamespace, isThreadGroup bool, cgroupControllers map[string]string) *kernfs.Dentry {
// TODO(gvisor.dev/issue/164): Fail with ESRCH if task exited.
contents := map[string]*kernfs.Dentry{
- "auxv": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &auxvData{task: task}),
- "cmdline": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &cmdlineData{task: task, arg: cmdlineDataArg}),
- "comm": newComm(task, inoGen.NextIno(), 0444),
- "environ": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &cmdlineData{task: task, arg: environDataArg}),
- "exe": newExeSymlink(task, inoGen.NextIno()),
- "fd": newFDDirInode(task, inoGen),
- "fdinfo": newFDInfoDirInode(task, inoGen),
- "gid_map": newTaskOwnedFile(task, inoGen.NextIno(), 0644, &idMapData{task: task, gids: true}),
- "io": newTaskOwnedFile(task, inoGen.NextIno(), 0400, newIO(task, isThreadGroup)),
- "maps": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &mapsData{task: task}),
- "mountinfo": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &mountInfoData{task: task}),
- "mounts": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &mountsData{task: task}),
- "net": newTaskNetDir(task, inoGen),
- "ns": newTaskOwnedDir(task, inoGen.NextIno(), 0511, map[string]*kernfs.Dentry{
- "net": newNamespaceSymlink(task, inoGen.NextIno(), "net"),
- "pid": newNamespaceSymlink(task, inoGen.NextIno(), "pid"),
- "user": newNamespaceSymlink(task, inoGen.NextIno(), "user"),
+ "auxv": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &auxvData{task: task}),
+ "cmdline": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &cmdlineData{task: task, arg: cmdlineDataArg}),
+ "comm": fs.newComm(task, fs.NextIno(), 0444),
+ "environ": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &cmdlineData{task: task, arg: environDataArg}),
+ "exe": fs.newExeSymlink(task, fs.NextIno()),
+ "fd": fs.newFDDirInode(task),
+ "fdinfo": fs.newFDInfoDirInode(task),
+ "gid_map": fs.newTaskOwnedFile(task, fs.NextIno(), 0644, &idMapData{task: task, gids: true}),
+ "io": fs.newTaskOwnedFile(task, fs.NextIno(), 0400, newIO(task, isThreadGroup)),
+ "maps": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &mapsData{task: task}),
+ "mountinfo": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &mountInfoData{task: task}),
+ "mounts": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &mountsData{task: task}),
+ "net": fs.newTaskNetDir(task),
+ "ns": fs.newTaskOwnedDir(task, fs.NextIno(), 0511, map[string]*kernfs.Dentry{
+ "net": fs.newNamespaceSymlink(task, fs.NextIno(), "net"),
+ "pid": fs.newNamespaceSymlink(task, fs.NextIno(), "pid"),
+ "user": fs.newNamespaceSymlink(task, fs.NextIno(), "user"),
}),
- "oom_score": newTaskOwnedFile(task, inoGen.NextIno(), 0444, newStaticFile("0\n")),
- "oom_score_adj": newTaskOwnedFile(task, inoGen.NextIno(), 0644, &oomScoreAdj{task: task}),
- "smaps": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &smapsData{task: task}),
- "stat": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &taskStatData{task: task, pidns: pidns, tgstats: isThreadGroup}),
- "statm": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &statmData{task: task}),
- "status": newTaskOwnedFile(task, inoGen.NextIno(), 0444, &statusData{task: task, pidns: pidns}),
- "uid_map": newTaskOwnedFile(task, inoGen.NextIno(), 0644, &idMapData{task: task, gids: false}),
+ "oom_score": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, newStaticFile("0\n")),
+ "oom_score_adj": fs.newTaskOwnedFile(task, fs.NextIno(), 0644, &oomScoreAdj{task: task}),
+ "smaps": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &smapsData{task: task}),
+ "stat": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &taskStatData{task: task, pidns: pidns, tgstats: isThreadGroup}),
+ "statm": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &statmData{task: task}),
+ "status": fs.newTaskOwnedFile(task, fs.NextIno(), 0444, &statusData{task: task, pidns: pidns}),
+ "uid_map": fs.newTaskOwnedFile(task, fs.NextIno(), 0644, &idMapData{task: task, gids: false}),
}
if isThreadGroup {
- contents["task"] = newSubtasks(task, pidns, inoGen, cgroupControllers)
+ contents["task"] = fs.newSubtasks(task, pidns, cgroupControllers)
}
if len(cgroupControllers) > 0 {
- contents["cgroup"] = newTaskOwnedFile(task, inoGen.NextIno(), 0444, newCgroupData(cgroupControllers))
+ contents["cgroup"] = fs.newTaskOwnedFile(task, fs.NextIno(), 0444, newCgroupData(cgroupControllers))
}
taskInode := &taskInode{task: task}
// Note: credentials are overridden by taskOwnedInode.
- taskInode.InodeAttrs.Init(task.Credentials(), inoGen.NextIno(), linux.ModeDirectory|0555)
+ taskInode.InodeAttrs.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0555)
inode := &taskOwnedInode{Inode: taskInode, owner: task}
dentry := &kernfs.Dentry{}
@@ -126,9 +126,9 @@ type taskOwnedInode struct {
var _ kernfs.Inode = (*taskOwnedInode)(nil)
-func newTaskOwnedFile(task *kernel.Task, ino uint64, perm linux.FileMode, inode dynamicInode) *kernfs.Dentry {
+func (fs *filesystem) newTaskOwnedFile(task *kernel.Task, ino uint64, perm linux.FileMode, inode dynamicInode) *kernfs.Dentry {
// Note: credentials are overridden by taskOwnedInode.
- inode.Init(task.Credentials(), ino, inode, perm)
+ inode.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, ino, inode, perm)
taskInode := &taskOwnedInode{Inode: inode, owner: task}
d := &kernfs.Dentry{}
@@ -136,11 +136,11 @@ func newTaskOwnedFile(task *kernel.Task, ino uint64, perm linux.FileMode, inode
return d
}
-func newTaskOwnedDir(task *kernel.Task, ino uint64, perm linux.FileMode, children map[string]*kernfs.Dentry) *kernfs.Dentry {
+func (fs *filesystem) newTaskOwnedDir(task *kernel.Task, ino uint64, perm linux.FileMode, children map[string]*kernfs.Dentry) *kernfs.Dentry {
dir := &kernfs.StaticDirectory{}
// Note: credentials are overridden by taskOwnedInode.
- dir.Init(task.Credentials(), ino, perm)
+ dir.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, ino, perm)
inode := &taskOwnedInode{Inode: dir, owner: task}
d := &kernfs.Dentry{}
diff --git a/pkg/sentry/fsimpl/proc/task_fds.go b/pkg/sentry/fsimpl/proc/task_fds.go
index 8ad976073..44ccc9e4a 100644
--- a/pkg/sentry/fsimpl/proc/task_fds.go
+++ b/pkg/sentry/fsimpl/proc/task_fds.go
@@ -53,8 +53,8 @@ func taskFDExists(t *kernel.Task, fd int32) bool {
}
type fdDir struct {
- inoGen InoGenerator
- task *kernel.Task
+ fs *filesystem
+ task *kernel.Task
// When produceSymlinks is set, dirents produces for the FDs are reported
// as symlink. Otherwise, they are reported as regular files.
@@ -85,7 +85,7 @@ func (i *fdDir) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback, abs
dirent := vfs.Dirent{
Name: strconv.FormatUint(uint64(fd), 10),
Type: typ,
- Ino: i.inoGen.NextIno(),
+ Ino: i.fs.NextIno(),
NextOff: offset + 1,
}
if err := cb.Handle(dirent); err != nil {
@@ -110,15 +110,15 @@ type fdDirInode struct {
var _ kernfs.Inode = (*fdDirInode)(nil)
-func newFDDirInode(task *kernel.Task, inoGen InoGenerator) *kernfs.Dentry {
+func (fs *filesystem) newFDDirInode(task *kernel.Task) *kernfs.Dentry {
inode := &fdDirInode{
fdDir: fdDir{
- inoGen: inoGen,
+ fs: fs,
task: task,
produceSymlink: true,
},
}
- inode.InodeAttrs.Init(task.Credentials(), inoGen.NextIno(), linux.ModeDirectory|0555)
+ inode.InodeAttrs.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0555)
dentry := &kernfs.Dentry{}
dentry.Init(inode)
@@ -137,7 +137,7 @@ func (i *fdDirInode) Lookup(ctx context.Context, name string) (*vfs.Dentry, erro
if !taskFDExists(i.task, fd) {
return nil, syserror.ENOENT
}
- taskDentry := newFDSymlink(i.task, fd, i.inoGen.NextIno())
+ taskDentry := i.fs.newFDSymlink(i.task, fd, i.fs.NextIno())
return taskDentry.VFSDentry(), nil
}
@@ -186,12 +186,12 @@ type fdSymlink struct {
var _ kernfs.Inode = (*fdSymlink)(nil)
-func newFDSymlink(task *kernel.Task, fd int32, ino uint64) *kernfs.Dentry {
+func (fs *filesystem) newFDSymlink(task *kernel.Task, fd int32, ino uint64) *kernfs.Dentry {
inode := &fdSymlink{
task: task,
fd: fd,
}
- inode.Init(task.Credentials(), ino, linux.ModeSymlink|0777)
+ inode.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, ino, linux.ModeSymlink|0777)
d := &kernfs.Dentry{}
d.Init(inode)
@@ -234,14 +234,14 @@ type fdInfoDirInode struct {
var _ kernfs.Inode = (*fdInfoDirInode)(nil)
-func newFDInfoDirInode(task *kernel.Task, inoGen InoGenerator) *kernfs.Dentry {
+func (fs *filesystem) newFDInfoDirInode(task *kernel.Task) *kernfs.Dentry {
inode := &fdInfoDirInode{
fdDir: fdDir{
- inoGen: inoGen,
- task: task,
+ fs: fs,
+ task: task,
},
}
- inode.InodeAttrs.Init(task.Credentials(), inoGen.NextIno(), linux.ModeDirectory|0555)
+ inode.InodeAttrs.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0555)
dentry := &kernfs.Dentry{}
dentry.Init(inode)
@@ -264,7 +264,7 @@ func (i *fdInfoDirInode) Lookup(ctx context.Context, name string) (*vfs.Dentry,
task: i.task,
fd: fd,
}
- dentry := newTaskOwnedFile(i.task, i.inoGen.NextIno(), 0444, data)
+ dentry := i.fs.newTaskOwnedFile(i.task, i.fs.NextIno(), 0444, data)
return dentry.VFSDentry(), nil
}
diff --git a/pkg/sentry/fsimpl/proc/task_files.go b/pkg/sentry/fsimpl/proc/task_files.go
index 515f25327..2f297e48a 100644
--- a/pkg/sentry/fsimpl/proc/task_files.go
+++ b/pkg/sentry/fsimpl/proc/task_files.go
@@ -241,9 +241,9 @@ type commInode struct {
task *kernel.Task
}
-func newComm(task *kernel.Task, ino uint64, perm linux.FileMode) *kernfs.Dentry {
+func (fs *filesystem) newComm(task *kernel.Task, ino uint64, perm linux.FileMode) *kernfs.Dentry {
inode := &commInode{task: task}
- inode.DynamicBytesFile.Init(task.Credentials(), ino, &commData{task: task}, perm)
+ inode.DynamicBytesFile.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, ino, &commData{task: task}, perm)
d := &kernfs.Dentry{}
d.Init(inode)
@@ -596,9 +596,9 @@ type exeSymlink struct {
var _ kernfs.Inode = (*exeSymlink)(nil)
-func newExeSymlink(task *kernel.Task, ino uint64) *kernfs.Dentry {
+func (fs *filesystem) newExeSymlink(task *kernel.Task, ino uint64) *kernfs.Dentry {
inode := &exeSymlink{task: task}
- inode.Init(task.Credentials(), ino, linux.ModeSymlink|0777)
+ inode.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, ino, linux.ModeSymlink|0777)
d := &kernfs.Dentry{}
d.Init(inode)
@@ -729,7 +729,7 @@ type namespaceSymlink struct {
task *kernel.Task
}
-func newNamespaceSymlink(task *kernel.Task, ino uint64, ns string) *kernfs.Dentry {
+func (fs *filesystem) newNamespaceSymlink(task *kernel.Task, ino uint64, ns string) *kernfs.Dentry {
// Namespace symlinks should contain the namespace name and the inode number
// for the namespace instance, so for example user:[123456]. We currently fake
// the inode number by sticking the symlink inode in its place.
@@ -737,7 +737,7 @@ func newNamespaceSymlink(task *kernel.Task, ino uint64, ns string) *kernfs.Dentr
inode := &namespaceSymlink{task: task}
// Note: credentials are overridden by taskOwnedInode.
- inode.Init(task.Credentials(), ino, target)
+ inode.Init(task.Credentials(), linux.UNNAMED_MAJOR, fs.devMinor, ino, target)
taskInode := &taskOwnedInode{Inode: inode, owner: task}
d := &kernfs.Dentry{}
@@ -780,11 +780,11 @@ type namespaceInode struct {
var _ kernfs.Inode = (*namespaceInode)(nil)
// Init initializes a namespace inode.
-func (i *namespaceInode) Init(creds *auth.Credentials, ino uint64, perm linux.FileMode) {
+func (i *namespaceInode) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode) {
if perm&^linux.PermissionsMask != 0 {
panic(fmt.Sprintf("Only permission mask must be set: %x", perm&linux.PermissionsMask))
}
- i.InodeAttrs.Init(creds, ino, linux.ModeRegular|perm)
+ i.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.ModeRegular|perm)
}
// Open implements Inode.Open.
diff --git a/pkg/sentry/fsimpl/proc/task_net.go b/pkg/sentry/fsimpl/proc/task_net.go
index 9c329341a..6bde27376 100644
--- a/pkg/sentry/fsimpl/proc/task_net.go
+++ b/pkg/sentry/fsimpl/proc/task_net.go
@@ -37,7 +37,7 @@ import (
"gvisor.dev/gvisor/pkg/usermem"
)
-func newTaskNetDir(task *kernel.Task, inoGen InoGenerator) *kernfs.Dentry {
+func (fs *filesystem) newTaskNetDir(task *kernel.Task) *kernfs.Dentry {
k := task.Kernel()
pidns := task.PIDNamespace()
root := auth.NewRootCredentials(pidns.UserNamespace())
@@ -57,37 +57,37 @@ func newTaskNetDir(task *kernel.Task, inoGen InoGenerator) *kernfs.Dentry {
// TODO(gvisor.dev/issue/1833): Make sure file contents reflect the task
// network namespace.
contents = map[string]*kernfs.Dentry{
- "dev": newDentry(root, inoGen.NextIno(), 0444, &netDevData{stack: stack}),
- "snmp": newDentry(root, inoGen.NextIno(), 0444, &netSnmpData{stack: stack}),
+ "dev": fs.newDentry(root, fs.NextIno(), 0444, &netDevData{stack: stack}),
+ "snmp": fs.newDentry(root, fs.NextIno(), 0444, &netSnmpData{stack: stack}),
// The following files are simple stubs until they are implemented in
// netstack, if the file contains a header the stub is just the header
// otherwise it is an empty file.
- "arp": newDentry(root, inoGen.NextIno(), 0444, newStaticFile(arp)),
- "netlink": newDentry(root, inoGen.NextIno(), 0444, newStaticFile(netlink)),
- "netstat": newDentry(root, inoGen.NextIno(), 0444, &netStatData{}),
- "packet": newDentry(root, inoGen.NextIno(), 0444, newStaticFile(packet)),
- "protocols": newDentry(root, inoGen.NextIno(), 0444, newStaticFile(protocols)),
+ "arp": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(arp)),
+ "netlink": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(netlink)),
+ "netstat": fs.newDentry(root, fs.NextIno(), 0444, &netStatData{}),
+ "packet": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(packet)),
+ "protocols": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(protocols)),
// Linux sets psched values to: nsec per usec, psched tick in ns, 1000000,
// high res timer ticks per sec (ClockGetres returns 1ns resolution).
- "psched": newDentry(root, inoGen.NextIno(), 0444, newStaticFile(psched)),
- "ptype": newDentry(root, inoGen.NextIno(), 0444, newStaticFile(ptype)),
- "route": newDentry(root, inoGen.NextIno(), 0444, &netRouteData{stack: stack}),
- "tcp": newDentry(root, inoGen.NextIno(), 0444, &netTCPData{kernel: k}),
- "udp": newDentry(root, inoGen.NextIno(), 0444, &netUDPData{kernel: k}),
- "unix": newDentry(root, inoGen.NextIno(), 0444, &netUnixData{kernel: k}),
+ "psched": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(psched)),
+ "ptype": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(ptype)),
+ "route": fs.newDentry(root, fs.NextIno(), 0444, &netRouteData{stack: stack}),
+ "tcp": fs.newDentry(root, fs.NextIno(), 0444, &netTCPData{kernel: k}),
+ "udp": fs.newDentry(root, fs.NextIno(), 0444, &netUDPData{kernel: k}),
+ "unix": fs.newDentry(root, fs.NextIno(), 0444, &netUnixData{kernel: k}),
}
if stack.SupportsIPv6() {
- contents["if_inet6"] = newDentry(root, inoGen.NextIno(), 0444, &ifinet6{stack: stack})
- contents["ipv6_route"] = newDentry(root, inoGen.NextIno(), 0444, newStaticFile(""))
- contents["tcp6"] = newDentry(root, inoGen.NextIno(), 0444, &netTCP6Data{kernel: k})
- contents["udp6"] = newDentry(root, inoGen.NextIno(), 0444, newStaticFile(upd6))
+ contents["if_inet6"] = fs.newDentry(root, fs.NextIno(), 0444, &ifinet6{stack: stack})
+ contents["ipv6_route"] = fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(""))
+ contents["tcp6"] = fs.newDentry(root, fs.NextIno(), 0444, &netTCP6Data{kernel: k})
+ contents["udp6"] = fs.newDentry(root, fs.NextIno(), 0444, newStaticFile(upd6))
}
}
- return newTaskOwnedDir(task, inoGen.NextIno(), 0555, contents)
+ return fs.newTaskOwnedDir(task, fs.NextIno(), 0555, contents)
}
// ifinet6 implements vfs.DynamicBytesSource for /proc/net/if_inet6.
diff --git a/pkg/sentry/fsimpl/proc/tasks.go b/pkg/sentry/fsimpl/proc/tasks.go
index 5aeda8c9b..b51d43954 100644
--- a/pkg/sentry/fsimpl/proc/tasks.go
+++ b/pkg/sentry/fsimpl/proc/tasks.go
@@ -33,11 +33,6 @@ const (
threadSelfName = "thread-self"
)
-// InoGenerator generates unique inode numbers for a given filesystem.
-type InoGenerator interface {
- NextIno() uint64
-}
-
// tasksInode represents the inode for /proc/ directory.
//
// +stateify savable
@@ -48,8 +43,8 @@ type tasksInode struct {
kernfs.OrderedChildren
kernfs.AlwaysValid
- inoGen InoGenerator
- pidns *kernel.PIDNamespace
+ fs *filesystem
+ pidns *kernel.PIDNamespace
// '/proc/self' and '/proc/thread-self' have custom directory offsets in
// Linux. So handle them outside of OrderedChildren.
@@ -64,29 +59,29 @@ type tasksInode struct {
var _ kernfs.Inode = (*tasksInode)(nil)
-func newTasksInode(inoGen InoGenerator, k *kernel.Kernel, pidns *kernel.PIDNamespace, cgroupControllers map[string]string) (*tasksInode, *kernfs.Dentry) {
+func (fs *filesystem) newTasksInode(k *kernel.Kernel, pidns *kernel.PIDNamespace, cgroupControllers map[string]string) (*tasksInode, *kernfs.Dentry) {
root := auth.NewRootCredentials(pidns.UserNamespace())
contents := map[string]*kernfs.Dentry{
- "cpuinfo": newDentry(root, inoGen.NextIno(), 0444, newStaticFileSetStat(cpuInfoData(k))),
- "filesystems": newDentry(root, inoGen.NextIno(), 0444, &filesystemsData{}),
- "loadavg": newDentry(root, inoGen.NextIno(), 0444, &loadavgData{}),
- "sys": newSysDir(root, inoGen, k),
- "meminfo": newDentry(root, inoGen.NextIno(), 0444, &meminfoData{}),
- "mounts": kernfs.NewStaticSymlink(root, inoGen.NextIno(), "self/mounts"),
- "net": kernfs.NewStaticSymlink(root, inoGen.NextIno(), "self/net"),
- "stat": newDentry(root, inoGen.NextIno(), 0444, &statData{}),
- "uptime": newDentry(root, inoGen.NextIno(), 0444, &uptimeData{}),
- "version": newDentry(root, inoGen.NextIno(), 0444, &versionData{}),
+ "cpuinfo": fs.newDentry(root, fs.NextIno(), 0444, newStaticFileSetStat(cpuInfoData(k))),
+ "filesystems": fs.newDentry(root, fs.NextIno(), 0444, &filesystemsData{}),
+ "loadavg": fs.newDentry(root, fs.NextIno(), 0444, &loadavgData{}),
+ "sys": fs.newSysDir(root, k),
+ "meminfo": fs.newDentry(root, fs.NextIno(), 0444, &meminfoData{}),
+ "mounts": kernfs.NewStaticSymlink(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), "self/mounts"),
+ "net": kernfs.NewStaticSymlink(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), "self/net"),
+ "stat": fs.newDentry(root, fs.NextIno(), 0444, &statData{}),
+ "uptime": fs.newDentry(root, fs.NextIno(), 0444, &uptimeData{}),
+ "version": fs.newDentry(root, fs.NextIno(), 0444, &versionData{}),
}
inode := &tasksInode{
pidns: pidns,
- inoGen: inoGen,
- selfSymlink: newSelfSymlink(root, inoGen.NextIno(), pidns).VFSDentry(),
- threadSelfSymlink: newThreadSelfSymlink(root, inoGen.NextIno(), pidns).VFSDentry(),
+ fs: fs,
+ selfSymlink: fs.newSelfSymlink(root, fs.NextIno(), pidns).VFSDentry(),
+ threadSelfSymlink: fs.newThreadSelfSymlink(root, fs.NextIno(), pidns).VFSDentry(),
cgroupControllers: cgroupControllers,
}
- inode.InodeAttrs.Init(root, inoGen.NextIno(), linux.ModeDirectory|0555)
+ inode.InodeAttrs.Init(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0555)
dentry := &kernfs.Dentry{}
dentry.Init(inode)
@@ -118,7 +113,7 @@ func (i *tasksInode) Lookup(ctx context.Context, name string) (*vfs.Dentry, erro
return nil, syserror.ENOENT
}
- taskDentry := newTaskInode(i.inoGen, task, i.pidns, true, i.cgroupControllers)
+ taskDentry := i.fs.newTaskInode(task, i.pidns, true, i.cgroupControllers)
return taskDentry.VFSDentry(), nil
}
@@ -144,7 +139,7 @@ func (i *tasksInode) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback
dirent := vfs.Dirent{
Name: selfName,
Type: linux.DT_LNK,
- Ino: i.inoGen.NextIno(),
+ Ino: i.fs.NextIno(),
NextOff: offset + 1,
}
if err := cb.Handle(dirent); err != nil {
@@ -156,7 +151,7 @@ func (i *tasksInode) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback
dirent := vfs.Dirent{
Name: threadSelfName,
Type: linux.DT_LNK,
- Ino: i.inoGen.NextIno(),
+ Ino: i.fs.NextIno(),
NextOff: offset + 1,
}
if err := cb.Handle(dirent); err != nil {
@@ -189,7 +184,7 @@ func (i *tasksInode) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback
dirent := vfs.Dirent{
Name: strconv.FormatUint(uint64(tid), 10),
Type: linux.DT_DIR,
- Ino: i.inoGen.NextIno(),
+ Ino: i.fs.NextIno(),
NextOff: FIRST_PROCESS_ENTRY + 2 + int64(tid) + 1,
}
if err := cb.Handle(dirent); err != nil {
diff --git a/pkg/sentry/fsimpl/proc/tasks_files.go b/pkg/sentry/fsimpl/proc/tasks_files.go
index e5f13b69e..7d8983aa5 100644
--- a/pkg/sentry/fsimpl/proc/tasks_files.go
+++ b/pkg/sentry/fsimpl/proc/tasks_files.go
@@ -41,9 +41,9 @@ type selfSymlink struct {
var _ kernfs.Inode = (*selfSymlink)(nil)
-func newSelfSymlink(creds *auth.Credentials, ino uint64, pidns *kernel.PIDNamespace) *kernfs.Dentry {
+func (fs *filesystem) newSelfSymlink(creds *auth.Credentials, ino uint64, pidns *kernel.PIDNamespace) *kernfs.Dentry {
inode := &selfSymlink{pidns: pidns}
- inode.Init(creds, ino, linux.ModeSymlink|0777)
+ inode.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, ino, linux.ModeSymlink|0777)
d := &kernfs.Dentry{}
d.Init(inode)
@@ -83,9 +83,9 @@ type threadSelfSymlink struct {
var _ kernfs.Inode = (*threadSelfSymlink)(nil)
-func newThreadSelfSymlink(creds *auth.Credentials, ino uint64, pidns *kernel.PIDNamespace) *kernfs.Dentry {
+func (fs *filesystem) newThreadSelfSymlink(creds *auth.Credentials, ino uint64, pidns *kernel.PIDNamespace) *kernfs.Dentry {
inode := &threadSelfSymlink{pidns: pidns}
- inode.Init(creds, ino, linux.ModeSymlink|0777)
+ inode.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, ino, linux.ModeSymlink|0777)
d := &kernfs.Dentry{}
d.Init(inode)
diff --git a/pkg/sentry/fsimpl/proc/tasks_sys.go b/pkg/sentry/fsimpl/proc/tasks_sys.go
index 0e90e02fe..6dac2afa4 100644
--- a/pkg/sentry/fsimpl/proc/tasks_sys.go
+++ b/pkg/sentry/fsimpl/proc/tasks_sys.go
@@ -30,89 +30,89 @@ import (
)
// newSysDir returns the dentry corresponding to /proc/sys directory.
-func newSysDir(root *auth.Credentials, inoGen InoGenerator, k *kernel.Kernel) *kernfs.Dentry {
- return kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, map[string]*kernfs.Dentry{
- "kernel": kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, map[string]*kernfs.Dentry{
- "hostname": newDentry(root, inoGen.NextIno(), 0444, &hostnameData{}),
- "shmall": newDentry(root, inoGen.NextIno(), 0444, shmData(linux.SHMALL)),
- "shmmax": newDentry(root, inoGen.NextIno(), 0444, shmData(linux.SHMMAX)),
- "shmmni": newDentry(root, inoGen.NextIno(), 0444, shmData(linux.SHMMNI)),
+func (fs *filesystem) newSysDir(root *auth.Credentials, k *kernel.Kernel) *kernfs.Dentry {
+ return kernfs.NewStaticDir(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), 0555, map[string]*kernfs.Dentry{
+ "kernel": kernfs.NewStaticDir(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), 0555, map[string]*kernfs.Dentry{
+ "hostname": fs.newDentry(root, fs.NextIno(), 0444, &hostnameData{}),
+ "shmall": fs.newDentry(root, fs.NextIno(), 0444, shmData(linux.SHMALL)),
+ "shmmax": fs.newDentry(root, fs.NextIno(), 0444, shmData(linux.SHMMAX)),
+ "shmmni": fs.newDentry(root, fs.NextIno(), 0444, shmData(linux.SHMMNI)),
}),
- "vm": kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, map[string]*kernfs.Dentry{
- "mmap_min_addr": newDentry(root, inoGen.NextIno(), 0444, &mmapMinAddrData{k: k}),
- "overcommit_memory": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0\n")),
+ "vm": kernfs.NewStaticDir(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), 0555, map[string]*kernfs.Dentry{
+ "mmap_min_addr": fs.newDentry(root, fs.NextIno(), 0444, &mmapMinAddrData{k: k}),
+ "overcommit_memory": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0\n")),
}),
- "net": newSysNetDir(root, inoGen, k),
+ "net": fs.newSysNetDir(root, k),
})
}
// newSysNetDir returns the dentry corresponding to /proc/sys/net directory.
-func newSysNetDir(root *auth.Credentials, inoGen InoGenerator, k *kernel.Kernel) *kernfs.Dentry {
+func (fs *filesystem) newSysNetDir(root *auth.Credentials, k *kernel.Kernel) *kernfs.Dentry {
var contents map[string]*kernfs.Dentry
// TODO(gvisor.dev/issue/1833): Support for using the network stack in the
// network namespace of the calling process.
if stack := k.RootNetworkNamespace().Stack(); stack != nil {
contents = map[string]*kernfs.Dentry{
- "ipv4": kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, map[string]*kernfs.Dentry{
- "tcp_sack": newDentry(root, inoGen.NextIno(), 0644, &tcpSackData{stack: stack}),
+ "ipv4": kernfs.NewStaticDir(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), 0555, map[string]*kernfs.Dentry{
+ "tcp_sack": fs.newDentry(root, fs.NextIno(), 0644, &tcpSackData{stack: stack}),
// The following files are simple stubs until they are implemented in
// netstack, most of these files are configuration related. We use the
// value closest to the actual netstack behavior or any empty file, all
// of these files will have mode 0444 (read-only for all users).
- "ip_local_port_range": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("16000 65535")),
- "ip_local_reserved_ports": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("")),
- "ipfrag_time": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("30")),
- "ip_nonlocal_bind": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "ip_no_pmtu_disc": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("1")),
+ "ip_local_port_range": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("16000 65535")),
+ "ip_local_reserved_ports": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("")),
+ "ipfrag_time": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("30")),
+ "ip_nonlocal_bind": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "ip_no_pmtu_disc": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("1")),
// tcp_allowed_congestion_control tell the user what they are able to
// do as an unprivledged process so we leave it empty.
- "tcp_allowed_congestion_control": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("")),
- "tcp_available_congestion_control": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("reno")),
- "tcp_congestion_control": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("reno")),
+ "tcp_allowed_congestion_control": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("")),
+ "tcp_available_congestion_control": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("reno")),
+ "tcp_congestion_control": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("reno")),
// Many of the following stub files are features netstack doesn't
// support. The unsupported features return "0" to indicate they are
// disabled.
- "tcp_base_mss": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("1280")),
- "tcp_dsack": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_early_retrans": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_fack": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_fastopen": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_fastopen_key": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("")),
- "tcp_invalid_ratelimit": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_keepalive_intvl": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_keepalive_probes": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_keepalive_time": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("7200")),
- "tcp_mtu_probing": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_no_metrics_save": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("1")),
- "tcp_probe_interval": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_probe_threshold": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "tcp_retries1": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("3")),
- "tcp_retries2": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("15")),
- "tcp_rfc1337": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("1")),
- "tcp_slow_start_after_idle": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("1")),
- "tcp_synack_retries": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("5")),
- "tcp_syn_retries": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("3")),
- "tcp_timestamps": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("1")),
+ "tcp_base_mss": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("1280")),
+ "tcp_dsack": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_early_retrans": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_fack": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_fastopen": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_fastopen_key": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("")),
+ "tcp_invalid_ratelimit": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_keepalive_intvl": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_keepalive_probes": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_keepalive_time": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("7200")),
+ "tcp_mtu_probing": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_no_metrics_save": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("1")),
+ "tcp_probe_interval": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_probe_threshold": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "tcp_retries1": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("3")),
+ "tcp_retries2": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("15")),
+ "tcp_rfc1337": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("1")),
+ "tcp_slow_start_after_idle": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("1")),
+ "tcp_synack_retries": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("5")),
+ "tcp_syn_retries": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("3")),
+ "tcp_timestamps": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("1")),
}),
- "core": kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, map[string]*kernfs.Dentry{
- "default_qdisc": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("pfifo_fast")),
- "message_burst": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("10")),
- "message_cost": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("5")),
- "optmem_max": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("0")),
- "rmem_default": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("212992")),
- "rmem_max": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("212992")),
- "somaxconn": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("128")),
- "wmem_default": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("212992")),
- "wmem_max": newDentry(root, inoGen.NextIno(), 0444, newStaticFile("212992")),
+ "core": kernfs.NewStaticDir(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), 0555, map[string]*kernfs.Dentry{
+ "default_qdisc": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("pfifo_fast")),
+ "message_burst": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("10")),
+ "message_cost": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("5")),
+ "optmem_max": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("0")),
+ "rmem_default": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("212992")),
+ "rmem_max": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("212992")),
+ "somaxconn": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("128")),
+ "wmem_default": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("212992")),
+ "wmem_max": fs.newDentry(root, fs.NextIno(), 0444, newStaticFile("212992")),
}),
}
}
- return kernfs.NewStaticDir(root, inoGen.NextIno(), 0555, contents)
+ return kernfs.NewStaticDir(root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), 0555, contents)
}
// mmapMinAddrData implements vfs.DynamicBytesSource for
diff --git a/pkg/sentry/fsimpl/signalfd/BUILD b/pkg/sentry/fsimpl/signalfd/BUILD
new file mode 100644
index 000000000..067c1657f
--- /dev/null
+++ b/pkg/sentry/fsimpl/signalfd/BUILD
@@ -0,0 +1,20 @@
+load("//tools:defs.bzl", "go_library")
+
+package(licenses = ["notice"])
+
+go_library(
+ name = "signalfd",
+ srcs = ["signalfd.go"],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/binary",
+ "//pkg/context",
+ "//pkg/sentry/kernel",
+ "//pkg/sentry/vfs",
+ "//pkg/sync",
+ "//pkg/syserror",
+ "//pkg/usermem",
+ "//pkg/waiter",
+ ],
+)
diff --git a/pkg/sentry/fsimpl/signalfd/signalfd.go b/pkg/sentry/fsimpl/signalfd/signalfd.go
new file mode 100644
index 000000000..d29ef3f83
--- /dev/null
+++ b/pkg/sentry/fsimpl/signalfd/signalfd.go
@@ -0,0 +1,135 @@
+// Copyright 2019 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 signalfd
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/binary"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// SignalFileDescription implements FileDescriptionImpl for signal fds.
+type SignalFileDescription struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ // target is the original signal target task.
+ //
+ // The semantics here are a bit broken. Linux will always use current
+ // for all reads, regardless of where the signalfd originated. We can't
+ // do exactly that because we need to plumb the context through
+ // EventRegister in order to support proper blocking behavior. This
+ // will undoubtedly become very complicated quickly.
+ target *kernel.Task
+
+ // mu protects mask.
+ mu sync.Mutex
+
+ // mask is the signal mask. Protected by mu.
+ mask linux.SignalSet
+}
+
+var _ vfs.FileDescriptionImpl = (*SignalFileDescription)(nil)
+
+// New creates a new signal fd.
+func New(vfsObj *vfs.VirtualFilesystem, target *kernel.Task, mask linux.SignalSet, flags uint32) (*vfs.FileDescription, error) {
+ vd := vfsObj.NewAnonVirtualDentry("[signalfd]")
+ defer vd.DecRef()
+ sfd := &SignalFileDescription{
+ target: target,
+ mask: mask,
+ }
+ if err := sfd.vfsfd.Init(sfd, flags, vd.Mount(), vd.Dentry(), &vfs.FileDescriptionOptions{
+ UseDentryMetadata: true,
+ DenyPRead: true,
+ DenyPWrite: true,
+ }); err != nil {
+ return nil, err
+ }
+ return &sfd.vfsfd, nil
+}
+
+// Mask returns the signal mask.
+func (sfd *SignalFileDescription) Mask() linux.SignalSet {
+ sfd.mu.Lock()
+ defer sfd.mu.Unlock()
+ return sfd.mask
+}
+
+// SetMask sets the signal mask.
+func (sfd *SignalFileDescription) SetMask(mask linux.SignalSet) {
+ sfd.mu.Lock()
+ defer sfd.mu.Unlock()
+ sfd.mask = mask
+}
+
+// Read implements FileDescriptionImpl.Read.
+func (sfd *SignalFileDescription) Read(ctx context.Context, dst usermem.IOSequence, _ vfs.ReadOptions) (int64, error) {
+ // Attempt to dequeue relevant signals.
+ info, err := sfd.target.Sigtimedwait(sfd.Mask(), 0)
+ if err != nil {
+ // There must be no signal available.
+ return 0, syserror.ErrWouldBlock
+ }
+
+ // Copy out the signal info using the specified format.
+ var buf [128]byte
+ binary.Marshal(buf[:0], usermem.ByteOrder, &linux.SignalfdSiginfo{
+ Signo: uint32(info.Signo),
+ Errno: info.Errno,
+ Code: info.Code,
+ PID: uint32(info.Pid()),
+ UID: uint32(info.Uid()),
+ Status: info.Status(),
+ Overrun: uint32(info.Overrun()),
+ Addr: info.Addr(),
+ })
+ n, err := dst.CopyOut(ctx, buf[:])
+ return int64(n), err
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (sfd *SignalFileDescription) Readiness(mask waiter.EventMask) waiter.EventMask {
+ sfd.mu.Lock()
+ defer sfd.mu.Unlock()
+ if mask&waiter.EventIn != 0 && sfd.target.PendingSignals()&sfd.mask != 0 {
+ return waiter.EventIn // Pending signals.
+ }
+ return 0
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (sfd *SignalFileDescription) EventRegister(entry *waiter.Entry, _ waiter.EventMask) {
+ sfd.mu.Lock()
+ defer sfd.mu.Unlock()
+ // Register for the signal set; ignore the passed events.
+ sfd.target.SignalRegister(entry, waiter.EventMask(sfd.mask))
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (sfd *SignalFileDescription) EventUnregister(entry *waiter.Entry) {
+ // Unregister the original entry.
+ sfd.target.SignalUnregister(entry)
+}
+
+// Release implements FileDescriptionImpl.Release()
+func (sfd *SignalFileDescription) Release() {}
diff --git a/pkg/sentry/fsimpl/sockfs/BUILD b/pkg/sentry/fsimpl/sockfs/BUILD
index 52084ddb5..9453277b8 100644
--- a/pkg/sentry/fsimpl/sockfs/BUILD
+++ b/pkg/sentry/fsimpl/sockfs/BUILD
@@ -9,6 +9,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/fspath",
"//pkg/sentry/fsimpl/kernfs",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/vfs",
diff --git a/pkg/sentry/fsimpl/sockfs/sockfs.go b/pkg/sentry/fsimpl/sockfs/sockfs.go
index 239a9f4b4..ee0828a15 100644
--- a/pkg/sentry/fsimpl/sockfs/sockfs.go
+++ b/pkg/sentry/fsimpl/sockfs/sockfs.go
@@ -16,8 +16,11 @@
package sockfs
import (
+ "fmt"
+
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
@@ -41,19 +44,42 @@ func (filesystemType) Name() string {
return "sockfs"
}
+type filesystem struct {
+ kernfs.Filesystem
+
+ devMinor uint32
+}
+
// 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 := &kernfs.Filesystem{}
- fs.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
- return fs.VFSFilesystem()
+func NewFilesystem(vfsObj *vfs.VirtualFilesystem) (*vfs.Filesystem, error) {
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, err
+ }
+ fs := &filesystem{
+ devMinor: devMinor,
+ }
+ fs.Filesystem.VFSFilesystem().Init(vfsObj, filesystemType{}, fs)
+ return fs.Filesystem.VFSFilesystem(), nil
+}
+
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release() {
+ fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release()
+}
+
+// PrependPath implements vfs.FilesystemImpl.PrependPath.
+func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDentry, b *fspath.Builder) error {
+ inode := vd.Dentry().Impl().(*kernfs.Dentry).Inode().(*inode)
+ b.PrependComponent(fmt.Sprintf("socket:[%d]", inode.InodeAttrs.Ino()))
+ return vfs.PrependPathSyntheticError{}
}
// inode implements kernfs.Inode.
-//
-// TODO(gvisor.dev/issue/1193): Device numbers.
type inode struct {
kernfs.InodeNotDirectory
kernfs.InodeNotSymlink
@@ -67,11 +93,15 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr
}
// NewDentry constructs and returns a sockfs dentry.
-func NewDentry(creds *auth.Credentials, ino uint64) *vfs.Dentry {
+//
+// Preconditions: mnt.Filesystem() must have been returned by NewFilesystem().
+func NewDentry(creds *auth.Credentials, mnt *vfs.Mount) *vfs.Dentry {
+ fs := mnt.Filesystem().Impl().(*filesystem)
+
// File mode matches net/socket.c:sock_alloc.
filemode := linux.FileMode(linux.S_IFSOCK | 0600)
i := &inode{}
- i.Init(creds, ino, filemode)
+ i.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.Filesystem.NextIno(), filemode)
d := &kernfs.Dentry{}
d.Init(i)
diff --git a/pkg/sentry/fsimpl/sys/sys.go b/pkg/sentry/fsimpl/sys/sys.go
index 00f7d6214..0af373604 100644
--- a/pkg/sentry/fsimpl/sys/sys.go
+++ b/pkg/sentry/fsimpl/sys/sys.go
@@ -37,6 +37,8 @@ type FilesystemType struct{}
// filesystem implements vfs.FilesystemImpl.
type filesystem struct {
kernfs.Filesystem
+
+ devMinor uint32
}
// Name implements vfs.FilesystemType.Name.
@@ -46,7 +48,14 @@ 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{}
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+
+ fs := &filesystem{
+ devMinor: devMinor,
+ }
fs.VFSFilesystem().Init(vfsObj, &fsType, fs)
k := kernel.KernelFromContext(ctx)
maxCPUCores := k.ApplicationCores()
@@ -77,6 +86,12 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
return fs.VFSFilesystem(), root.VFSDentry(), nil
}
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release() {
+ fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release()
+}
+
// dir implements kernfs.Inode.
type dir struct {
kernfs.InodeAttrs
@@ -90,7 +105,7 @@ type dir struct {
func (fs *filesystem) newDir(creds *auth.Credentials, mode linux.FileMode, contents map[string]*kernfs.Dentry) *kernfs.Dentry {
d := &dir{}
- d.InodeAttrs.Init(creds, fs.NextIno(), linux.ModeDirectory|0755)
+ d.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0755)
d.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
d.dentry.Init(d)
@@ -127,7 +142,7 @@ func (c *cpuFile) Generate(ctx context.Context, buf *bytes.Buffer) error {
func (fs *filesystem) newCPUFile(creds *auth.Credentials, maxCores uint, mode linux.FileMode) *kernfs.Dentry {
c := &cpuFile{maxCores: maxCores}
- c.DynamicBytesFile.Init(creds, fs.NextIno(), c, mode)
+ c.DynamicBytesFile.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), c, mode)
d := &kernfs.Dentry{}
d.Init(c)
return d
diff --git a/pkg/sentry/fsimpl/timerfd/BUILD b/pkg/sentry/fsimpl/timerfd/BUILD
new file mode 100644
index 000000000..fbb02a271
--- /dev/null
+++ b/pkg/sentry/fsimpl/timerfd/BUILD
@@ -0,0 +1,17 @@
+load("//tools:defs.bzl", "go_library")
+
+licenses(["notice"])
+
+go_library(
+ name = "timerfd",
+ srcs = ["timerfd.go"],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/context",
+ "//pkg/sentry/kernel/time",
+ "//pkg/sentry/vfs",
+ "//pkg/syserror",
+ "//pkg/usermem",
+ "//pkg/waiter",
+ ],
+)
diff --git a/pkg/sentry/fsimpl/timerfd/timerfd.go b/pkg/sentry/fsimpl/timerfd/timerfd.go
new file mode 100644
index 000000000..60c92d626
--- /dev/null
+++ b/pkg/sentry/fsimpl/timerfd/timerfd.go
@@ -0,0 +1,143 @@
+// 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 timerfd implements timer fds.
+package timerfd
+
+import (
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/context"
+ ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// TimerFileDescription implements FileDescriptionImpl for timer fds. It also
+// implements ktime.TimerListener.
+type TimerFileDescription struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ events waiter.Queue
+ timer *ktime.Timer
+
+ // val is the number of timer expirations since the last successful
+ // call to PRead, or SetTime. val must be accessed using atomic memory
+ // operations.
+ val uint64
+}
+
+var _ vfs.FileDescriptionImpl = (*TimerFileDescription)(nil)
+var _ ktime.TimerListener = (*TimerFileDescription)(nil)
+
+// New returns a new timer fd.
+func New(vfsObj *vfs.VirtualFilesystem, clock ktime.Clock, flags uint32) (*vfs.FileDescription, error) {
+ vd := vfsObj.NewAnonVirtualDentry("[timerfd]")
+ defer vd.DecRef()
+ tfd := &TimerFileDescription{}
+ tfd.timer = ktime.NewTimer(clock, tfd)
+ if err := tfd.vfsfd.Init(tfd, flags, vd.Mount(), vd.Dentry(), &vfs.FileDescriptionOptions{
+ UseDentryMetadata: true,
+ DenyPRead: true,
+ DenyPWrite: true,
+ }); err != nil {
+ return nil, err
+ }
+ return &tfd.vfsfd, nil
+}
+
+// Read implements FileDescriptionImpl.Read.
+func (tfd *TimerFileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+ const sizeofUint64 = 8
+ if dst.NumBytes() < sizeofUint64 {
+ return 0, syserror.EINVAL
+ }
+ if val := atomic.SwapUint64(&tfd.val, 0); val != 0 {
+ var buf [sizeofUint64]byte
+ usermem.ByteOrder.PutUint64(buf[:], val)
+ if _, err := dst.CopyOut(ctx, buf[:]); err != nil {
+ // Linux does not undo consuming the number of
+ // expirations even if writing to userspace fails.
+ return 0, err
+ }
+ return sizeofUint64, nil
+ }
+ return 0, syserror.ErrWouldBlock
+}
+
+// Clock returns the timer fd's Clock.
+func (tfd *TimerFileDescription) Clock() ktime.Clock {
+ return tfd.timer.Clock()
+}
+
+// GetTime returns the associated Timer's setting and the time at which it was
+// observed.
+func (tfd *TimerFileDescription) GetTime() (ktime.Time, ktime.Setting) {
+ return tfd.timer.Get()
+}
+
+// SetTime atomically changes the associated Timer's setting, resets the number
+// of expirations to 0, and returns the previous setting and the time at which
+// it was observed.
+func (tfd *TimerFileDescription) SetTime(s ktime.Setting) (ktime.Time, ktime.Setting) {
+ return tfd.timer.SwapAnd(s, func() { atomic.StoreUint64(&tfd.val, 0) })
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (tfd *TimerFileDescription) Readiness(mask waiter.EventMask) waiter.EventMask {
+ var ready waiter.EventMask
+ if atomic.LoadUint64(&tfd.val) != 0 {
+ ready |= waiter.EventIn
+ }
+ return ready
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (tfd *TimerFileDescription) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ tfd.events.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (tfd *TimerFileDescription) EventUnregister(e *waiter.Entry) {
+ tfd.events.EventUnregister(e)
+}
+
+// PauseTimer pauses the associated Timer.
+func (tfd *TimerFileDescription) PauseTimer() {
+ tfd.timer.Pause()
+}
+
+// ResumeTimer resumes the associated Timer.
+func (tfd *TimerFileDescription) ResumeTimer() {
+ tfd.timer.Resume()
+}
+
+// Release implements FileDescriptionImpl.Release()
+func (tfd *TimerFileDescription) Release() {
+ tfd.timer.Destroy()
+}
+
+// Notify implements ktime.TimerListener.Notify.
+func (tfd *TimerFileDescription) Notify(exp uint64, setting ktime.Setting) (ktime.Setting, bool) {
+ atomic.AddUint64(&tfd.val, exp)
+ tfd.events.Notify(waiter.EventIn)
+ return ktime.Setting{}, false
+}
+
+// Destroy implements ktime.TimerListener.Destroy.
+func (tfd *TimerFileDescription) Destroy() {}
diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
index efc931468..405928bd0 100644
--- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go
+++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
@@ -63,6 +63,9 @@ type filesystem struct {
// clock is a realtime clock used to set timestamps in file operations.
clock time.Clock
+ // devMinor is the filesystem's minor device number. devMinor is immutable.
+ devMinor uint32
+
// mu serializes changes to the Dentry tree.
mu sync.RWMutex
@@ -96,11 +99,6 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
if memFileProvider == nil {
panic("MemoryFileProviderFromContext returned nil")
}
- clock := time.RealtimeClockFromContext(ctx)
- fs := filesystem{
- memFile: memFileProvider.MemoryFile(),
- clock: clock,
- }
rootFileType := uint16(linux.S_IFDIR)
newFSType := vfs.FilesystemType(&fstype)
@@ -114,6 +112,16 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
}
}
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+ clock := time.RealtimeClockFromContext(ctx)
+ fs := filesystem{
+ memFile: memFileProvider.MemoryFile(),
+ clock: clock,
+ devMinor: devMinor,
+ }
fs.vfsfs.Init(vfsObj, newFSType, &fs)
var root *dentry
@@ -125,6 +133,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
case linux.S_IFDIR:
root = &fs.newDirectory(creds, 01777).dentry
default:
+ fs.vfsfs.DecRef()
return nil, nil, fmt.Errorf("invalid tmpfs root file type: %#o", rootFileType)
}
return &fs.vfsfs, &root.vfsd, nil
@@ -132,6 +141,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
// Release implements vfs.FilesystemImpl.Release.
func (fs *filesystem) Release() {
+ fs.vfsfs.VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
}
// dentry implements vfs.DentryImpl.
@@ -188,8 +198,8 @@ func (d *dentry) DecRef() {
// inode represents a filesystem object.
type inode struct {
- // clock is a realtime clock used to set timestamps in file operations.
- clock time.Clock
+ // fs is the owning filesystem. fs is immutable.
+ fs *filesystem
// refs is a reference count. refs is accessed using atomic memory
// operations.
@@ -232,7 +242,7 @@ func (i *inode) init(impl interface{}, fs *filesystem, creds *auth.Credentials,
if mode.FileType() == 0 {
panic("file type is required in FileMode")
}
- i.clock = fs.clock
+ i.fs = fs
i.refs = 1
i.mode = uint32(mode)
i.uid = uint32(creds.EffectiveKUID)
@@ -327,7 +337,8 @@ func (i *inode) statTo(stat *linux.Statx) {
stat.Atime = linux.NsecToStatxTimestamp(i.atime)
stat.Ctime = linux.NsecToStatxTimestamp(i.ctime)
stat.Mtime = linux.NsecToStatxTimestamp(i.mtime)
- // TODO(gvisor.dev/issue/1197): Device number.
+ stat.DevMajor = linux.UNNAMED_MAJOR
+ stat.DevMinor = i.fs.devMinor
switch impl := i.impl.(type) {
case *regularFile:
stat.Mask |= linux.STATX_SIZE | linux.STATX_BLOCKS
@@ -401,7 +412,7 @@ func (i *inode) setStat(ctx context.Context, creds *auth.Credentials, stat *linu
return syserror.EINVAL
}
}
- now := i.clock.Now().Nanoseconds()
+ now := i.fs.clock.Now().Nanoseconds()
if mask&linux.STATX_ATIME != 0 {
if stat.Atime.Nsec == linux.UTIME_NOW {
atomic.StoreInt64(&i.atime, now)
@@ -518,7 +529,7 @@ func (i *inode) touchAtime(mnt *vfs.Mount) {
if err := mnt.CheckBeginWrite(); err != nil {
return
}
- now := i.clock.Now().Nanoseconds()
+ now := i.fs.clock.Now().Nanoseconds()
i.mu.Lock()
atomic.StoreInt64(&i.atime, now)
i.mu.Unlock()
@@ -527,7 +538,7 @@ func (i *inode) touchAtime(mnt *vfs.Mount) {
// Preconditions: The caller has called vfs.Mount.CheckBeginWrite().
func (i *inode) touchCtime() {
- now := i.clock.Now().Nanoseconds()
+ now := i.fs.clock.Now().Nanoseconds()
i.mu.Lock()
atomic.StoreInt64(&i.ctime, now)
i.mu.Unlock()
@@ -535,7 +546,7 @@ func (i *inode) touchCtime() {
// Preconditions: The caller has called vfs.Mount.CheckBeginWrite().
func (i *inode) touchCMtime() {
- now := i.clock.Now().Nanoseconds()
+ now := i.fs.clock.Now().Nanoseconds()
i.mu.Lock()
atomic.StoreInt64(&i.mtime, now)
atomic.StoreInt64(&i.ctime, now)
@@ -545,7 +556,7 @@ func (i *inode) touchCMtime() {
// Preconditions: The caller has called vfs.Mount.CheckBeginWrite() and holds
// inode.mu.
func (i *inode) touchCMtimeLocked() {
- now := i.clock.Now().Nanoseconds()
+ now := i.fs.clock.Now().Nanoseconds()
atomic.StoreInt64(&i.mtime, now)
atomic.StoreInt64(&i.ctime, now)
}