diff options
Diffstat (limited to 'pkg/sentry/vfs')
-rw-r--r-- | pkg/sentry/vfs/README.md | 4 | ||||
-rw-r--r-- | pkg/sentry/vfs/dentry.go | 150 | ||||
-rw-r--r-- | pkg/sentry/vfs/file_description.go | 33 | ||||
-rw-r--r-- | pkg/sentry/vfs/file_description_impl_util_test.go | 2 | ||||
-rw-r--r-- | pkg/sentry/vfs/filesystem.go | 20 | ||||
-rw-r--r-- | pkg/sentry/vfs/filesystem_type.go | 10 | ||||
-rw-r--r-- | pkg/sentry/vfs/mount.go | 359 | ||||
-rw-r--r-- | pkg/sentry/vfs/mount_test.go | 34 | ||||
-rw-r--r-- | pkg/sentry/vfs/mount_unsafe.go | 60 | ||||
-rw-r--r-- | pkg/sentry/vfs/resolving_path.go | 16 | ||||
-rw-r--r-- | pkg/sentry/vfs/syscalls.go | 4 | ||||
-rw-r--r-- | pkg/sentry/vfs/testutil.go | 12 | ||||
-rw-r--r-- | pkg/sentry/vfs/vfs.go | 19 |
13 files changed, 501 insertions, 222 deletions
diff --git a/pkg/sentry/vfs/README.md b/pkg/sentry/vfs/README.md index 7847854bc..9aa133bcb 100644 --- a/pkg/sentry/vfs/README.md +++ b/pkg/sentry/vfs/README.md @@ -39,8 +39,8 @@ Mount references are held by: - Mount: Each referenced Mount holds a reference on its parent, which is the mount containing its mount point. -- VirtualFilesystem: A reference is held on all Mounts that are attached - (reachable by Mount traversal). +- VirtualFilesystem: A reference is held on each Mount that has not been + umounted. MountNamespace and FileDescription references are held by users of VFS. The expectation is that each `kernel.Task` holds a reference on its corresponding diff --git a/pkg/sentry/vfs/dentry.go b/pkg/sentry/vfs/dentry.go index 45912fc58..40f4c1d09 100644 --- a/pkg/sentry/vfs/dentry.go +++ b/pkg/sentry/vfs/dentry.go @@ -16,6 +16,7 @@ package vfs import ( "fmt" + "sync" "sync/atomic" "gvisor.dev/gvisor/pkg/syserror" @@ -50,7 +51,7 @@ import ( // and not inodes. Furthermore, when parties outside the scope of VFS can // rename inodes on such filesystems, VFS generally cannot "follow" the rename, // both due to synchronization issues and because it may not even be able to -// name the destination path; this implies that it would in fact be *incorrect* +// name the destination path; this implies that it would in fact be incorrect // for Dentries to be associated with inodes on such filesystems. Consequently, // operations that are inode operations in Linux are FilesystemImpl methods // and/or FileDescriptionImpl methods in gVisor's VFS. Filesystems that do @@ -84,6 +85,9 @@ type Dentry struct { // mounts is accessed using atomic memory operations. mounts uint32 + // mu synchronizes disowning and mounting over this Dentry. + mu sync.Mutex + // children are child Dentries. children map[string]*Dentry @@ -114,7 +118,7 @@ func (d *Dentry) Impl() DentryImpl { type DentryImpl interface { // IncRef increments the Dentry's reference count. A Dentry with a non-zero // reference count must remain coherent with the state of the filesystem. - IncRef(fs *Filesystem) + IncRef() // TryIncRef increments the Dentry's reference count and returns true. If // the Dentry's reference count is zero, TryIncRef may do nothing and @@ -122,10 +126,10 @@ type DentryImpl interface { // guarantee that the Dentry is coherent with the state of the filesystem.) // // TryIncRef does not require that a reference is held on the Dentry. - TryIncRef(fs *Filesystem) bool + TryIncRef() bool // DecRef decrements the Dentry's reference count. - DecRef(fs *Filesystem) + DecRef() } // IsDisowned returns true if d is disowned. @@ -142,16 +146,20 @@ func (d *Dentry) isMounted() bool { return atomic.LoadUint32(&d.mounts) != 0 } -func (d *Dentry) incRef(fs *Filesystem) { - d.impl.IncRef(fs) +// IncRef increments d's reference count. +func (d *Dentry) IncRef() { + d.impl.IncRef() } -func (d *Dentry) tryIncRef(fs *Filesystem) bool { - return d.impl.TryIncRef(fs) +// TryIncRef increments d's reference count and returns true. If d's reference +// count is zero, TryIncRef may instead do nothing and return false. +func (d *Dentry) TryIncRef() bool { + return d.impl.TryIncRef() } -func (d *Dentry) decRef(fs *Filesystem) { - d.impl.DecRef(fs) +// DecRef decrements d's reference count. +func (d *Dentry) DecRef() { + d.impl.DecRef() } // These functions are exported so that filesystem implementations can use @@ -228,36 +236,48 @@ func (vfs *VirtualFilesystem) PrepareDeleteDentry(mntns *MountNamespace, d *Dent panic("d is already disowned") } } - vfs.mountMu.RLock() - if _, ok := mntns.mountpoints[d]; ok { - vfs.mountMu.RUnlock() + vfs.mountMu.Lock() + if mntns.mountpoints[d] != 0 { + vfs.mountMu.Unlock() return syserror.EBUSY } - // Return with vfs.mountMu locked, which will be unlocked by - // AbortDeleteDentry or CommitDeleteDentry. + d.mu.Lock() + vfs.mountMu.Unlock() + // Return with d.mu locked to block attempts to mount over it; it will be + // unlocked by AbortDeleteDentry or CommitDeleteDentry. return nil } // AbortDeleteDentry must be called after PrepareDeleteDentry if the deletion // fails. -func (vfs *VirtualFilesystem) AbortDeleteDentry() { - vfs.mountMu.RUnlock() +func (vfs *VirtualFilesystem) AbortDeleteDentry(d *Dentry) { + d.mu.Unlock() } // CommitDeleteDentry must be called after the file represented by d is // deleted, and causes d to become disowned. // +// CommitDeleteDentry is a mutator of d and d.Parent(). +// // Preconditions: PrepareDeleteDentry was previously called on d. func (vfs *VirtualFilesystem) CommitDeleteDentry(d *Dentry) { - delete(d.parent.children, d.name) + if d.parent != nil { + delete(d.parent.children, d.name) + } d.setDisowned() - // TODO: lazily unmount mounts at d - vfs.mountMu.RUnlock() + d.mu.Unlock() + if d.isMounted() { + vfs.forgetDisownedMountpoint(d) + } } // DeleteDentry combines PrepareDeleteDentry and CommitDeleteDentry, as // appropriate for in-memory filesystems that don't need to ensure that some // external state change succeeds before committing the deletion. +// +// DeleteDentry is a mutator of d and d.Parent(). +// +// Preconditions: d is a child Dentry. func (vfs *VirtualFilesystem) DeleteDentry(mntns *MountNamespace, d *Dentry) error { if err := vfs.PrepareDeleteDentry(mntns, d); err != nil { return err @@ -266,6 +286,27 @@ func (vfs *VirtualFilesystem) DeleteDentry(mntns *MountNamespace, d *Dentry) err return nil } +// ForceDeleteDentry causes d to become disowned. It should only be used in +// cases where VFS has no ability to stop the deletion (e.g. d represents the +// local state of a file on a remote filesystem on which the file has already +// been deleted). +// +// ForceDeleteDentry is a mutator of d and d.Parent(). +// +// Preconditions: d is a child Dentry. +func (vfs *VirtualFilesystem) ForceDeleteDentry(d *Dentry) { + if checkInvariants { + if d.parent == nil { + panic("d is independent") + } + if d.IsDisowned() { + panic("d is already disowned") + } + } + d.mu.Lock() + vfs.CommitDeleteDentry(d) +} + // PrepareRenameDentry must be called before attempting to rename the file // represented by from. If to is not nil, it represents the file that will be // replaced or exchanged by the rename. If PrepareRenameDentry succeeds, the @@ -291,18 +332,21 @@ func (vfs *VirtualFilesystem) PrepareRenameDentry(mntns *MountNamespace, from, t } } } - vfs.mountMu.RLock() - if _, ok := mntns.mountpoints[from]; ok { - vfs.mountMu.RUnlock() + vfs.mountMu.Lock() + if mntns.mountpoints[from] != 0 { + vfs.mountMu.Unlock() return syserror.EBUSY } if to != nil { - if _, ok := mntns.mountpoints[to]; ok { - vfs.mountMu.RUnlock() + if mntns.mountpoints[to] != 0 { + vfs.mountMu.Unlock() return syserror.EBUSY } + to.mu.Lock() } - // Return with vfs.mountMu locked, which will be unlocked by + from.mu.Lock() + vfs.mountMu.Unlock() + // Return with from.mu and to.mu locked, which will be unlocked by // AbortRenameDentry, CommitRenameReplaceDentry, or // CommitRenameExchangeDentry. return nil @@ -310,38 +354,76 @@ func (vfs *VirtualFilesystem) PrepareRenameDentry(mntns *MountNamespace, from, t // AbortRenameDentry must be called after PrepareRenameDentry if the rename // fails. -func (vfs *VirtualFilesystem) AbortRenameDentry() { - vfs.mountMu.RUnlock() +func (vfs *VirtualFilesystem) AbortRenameDentry(from, to *Dentry) { + from.mu.Unlock() + if to != nil { + to.mu.Unlock() + } } // CommitRenameReplaceDentry must be called after the file represented by from // is renamed without RENAME_EXCHANGE. If to is not nil, it represents the file // that was replaced by from. // +// CommitRenameReplaceDentry is a mutator of from, to, from.Parent(), and +// to.Parent(). +// // Preconditions: PrepareRenameDentry was previously called on from and to. // newParent.Child(newName) == to. func (vfs *VirtualFilesystem) CommitRenameReplaceDentry(from, newParent *Dentry, newName string, to *Dentry) { - if to != nil { - to.setDisowned() - // TODO: lazily unmount mounts at d - } if newParent.children == nil { newParent.children = make(map[string]*Dentry) } newParent.children[newName] = from from.parent = newParent from.name = newName - vfs.mountMu.RUnlock() + from.mu.Unlock() + if to != nil { + to.setDisowned() + to.mu.Unlock() + if to.isMounted() { + vfs.forgetDisownedMountpoint(to) + } + } } // CommitRenameExchangeDentry must be called after the files represented by // from and to are exchanged by rename(RENAME_EXCHANGE). // +// CommitRenameExchangeDentry is a mutator of from, to, from.Parent(), and +// to.Parent(). +// // Preconditions: PrepareRenameDentry was previously called on from and to. func (vfs *VirtualFilesystem) CommitRenameExchangeDentry(from, to *Dentry) { from.parent, to.parent = to.parent, from.parent from.name, to.name = to.name, from.name from.parent.children[from.name] = from to.parent.children[to.name] = to - vfs.mountMu.RUnlock() + from.mu.Unlock() + to.mu.Unlock() +} + +// forgetDisownedMountpoint is called when a mount point is deleted to umount +// all mounts using it in all other mount namespaces. +// +// forgetDisownedMountpoint is analogous to Linux's +// fs/namespace.c:__detach_mounts(). +func (vfs *VirtualFilesystem) forgetDisownedMountpoint(d *Dentry) { + var ( + vdsToDecRef []VirtualDentry + mountsToDecRef []*Mount + ) + vfs.mountMu.Lock() + vfs.mounts.seq.BeginWrite() + for mnt := range vfs.mountpoints[d] { + vdsToDecRef, mountsToDecRef = vfs.umountRecursiveLocked(mnt, &umountRecursiveOptions{}, vdsToDecRef, mountsToDecRef) + } + vfs.mounts.seq.EndWrite() + vfs.mountMu.Unlock() + for _, vd := range vdsToDecRef { + vd.DecRef() + } + for _, mnt := range mountsToDecRef { + mnt.DecRef() + } } diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go index 3a9665800..34007eb57 100644 --- a/pkg/sentry/vfs/file_description.go +++ b/pkg/sentry/vfs/file_description.go @@ -47,15 +47,14 @@ type FileDescription struct { impl FileDescriptionImpl } -// Init must be called before first use of fd. It takes references on mnt and -// d. +// Init must be called before first use of fd. It takes ownership of references +// on mnt and d held by the caller. func (fd *FileDescription) Init(impl FileDescriptionImpl, mnt *Mount, d *Dentry) { fd.refs = 1 fd.vd = VirtualDentry{ mount: mnt, dentry: d, } - fd.vd.IncRef() fd.impl = impl } @@ -64,6 +63,18 @@ func (fd *FileDescription) Impl() FileDescriptionImpl { return fd.impl } +// Mount returns the mount on which fd was opened. It does not take a reference +// on the returned Mount. +func (fd *FileDescription) Mount() *Mount { + return fd.vd.mount +} + +// Dentry returns the dentry at which fd was opened. It does not take a +// reference on the returned Dentry. +func (fd *FileDescription) Dentry() *Dentry { + return fd.vd.dentry +} + // VirtualDentry returns the location at which fd was opened. It does not take // a reference on the returned VirtualDentry. func (fd *FileDescription) VirtualDentry() VirtualDentry { @@ -75,6 +86,22 @@ func (fd *FileDescription) IncRef() { atomic.AddInt64(&fd.refs, 1) } +// TryIncRef increments fd's reference count and returns true. If fd's +// reference count is already zero, TryIncRef does nothing and returns false. +// +// TryIncRef does not require that a reference is held on fd. +func (fd *FileDescription) TryIncRef() bool { + for { + refs := atomic.LoadInt64(&fd.refs) + if refs <= 0 { + return false + } + if atomic.CompareAndSwapInt64(&fd.refs, refs, refs+1) { + return true + } + } +} + // DecRef decrements fd's reference count. func (fd *FileDescription) DecRef() { if refs := atomic.AddInt64(&fd.refs, -1); refs == 0 { diff --git a/pkg/sentry/vfs/file_description_impl_util_test.go b/pkg/sentry/vfs/file_description_impl_util_test.go index 511b829fc..a5561dcbe 100644 --- a/pkg/sentry/vfs/file_description_impl_util_test.go +++ b/pkg/sentry/vfs/file_description_impl_util_test.go @@ -90,7 +90,7 @@ func TestGenCountFD(t *testing.T) { vfsObj := New() // vfs.New() vfsObj.MustRegisterFilesystemType("testfs", FDTestFilesystemType{}) - mntns, err := vfsObj.NewMountNamespace(ctx, creds, "", "testfs", &NewFilesystemOptions{}) + mntns, err := vfsObj.NewMountNamespace(ctx, creds, "", "testfs", &GetFilesystemOptions{}) if err != nil { t.Fatalf("failed to create testfs root mount: %v", err) } diff --git a/pkg/sentry/vfs/filesystem.go b/pkg/sentry/vfs/filesystem.go index 7a074b718..76ff8cf51 100644 --- a/pkg/sentry/vfs/filesystem.go +++ b/pkg/sentry/vfs/filesystem.go @@ -33,29 +33,41 @@ type Filesystem struct { // operations. refs int64 + // vfs is the VirtualFilesystem that uses this Filesystem. vfs is + // immutable. + vfs *VirtualFilesystem + // impl is the FilesystemImpl associated with this Filesystem. impl is // immutable. This should be the last field in Dentry. impl FilesystemImpl } // Init must be called before first use of fs. -func (fs *Filesystem) Init(impl FilesystemImpl) { +func (fs *Filesystem) Init(vfsObj *VirtualFilesystem, impl FilesystemImpl) { fs.refs = 1 + fs.vfs = vfsObj fs.impl = impl } +// VirtualFilesystem returns the containing VirtualFilesystem. +func (fs *Filesystem) VirtualFilesystem() *VirtualFilesystem { + return fs.vfs +} + // Impl returns the FilesystemImpl associated with fs. func (fs *Filesystem) Impl() FilesystemImpl { return fs.impl } -func (fs *Filesystem) incRef() { +// IncRef increments fs' reference count. +func (fs *Filesystem) IncRef() { if atomic.AddInt64(&fs.refs, 1) <= 1 { - panic("Filesystem.incRef() called without holding a reference") + panic("Filesystem.IncRef() called without holding a reference") } } -func (fs *Filesystem) decRef() { +// DecRef decrements fs' reference count. +func (fs *Filesystem) DecRef() { if refs := atomic.AddInt64(&fs.refs, -1); refs == 0 { fs.impl.Release() } else if refs < 0 { diff --git a/pkg/sentry/vfs/filesystem_type.go b/pkg/sentry/vfs/filesystem_type.go index f401ad7f3..c335e206d 100644 --- a/pkg/sentry/vfs/filesystem_type.go +++ b/pkg/sentry/vfs/filesystem_type.go @@ -25,21 +25,21 @@ import ( // // FilesystemType is analogous to Linux's struct file_system_type. type FilesystemType interface { - // NewFilesystem returns a Filesystem configured by the given options, + // GetFilesystem returns a Filesystem configured by the given options, // along with its mount root. A reference is taken on the returned // Filesystem and Dentry. - NewFilesystem(ctx context.Context, creds *auth.Credentials, source string, opts NewFilesystemOptions) (*Filesystem, *Dentry, error) + GetFilesystem(ctx context.Context, vfsObj *VirtualFilesystem, creds *auth.Credentials, source string, opts GetFilesystemOptions) (*Filesystem, *Dentry, error) } -// NewFilesystemOptions contains options to FilesystemType.NewFilesystem. -type NewFilesystemOptions struct { +// GetFilesystemOptions contains options to FilesystemType.GetFilesystem. +type GetFilesystemOptions struct { // Data is the string passed as the 5th argument to mount(2), which is // usually a comma-separated list of filesystem-specific mount options. Data string // InternalData holds opaque FilesystemType-specific data. There is // intentionally no way for applications to specify InternalData; if it is - // not nil, the call to NewFilesystem originates from within the sentry. + // not nil, the call to GetFilesystem originates from within the sentry. InternalData interface{} } diff --git a/pkg/sentry/vfs/mount.go b/pkg/sentry/vfs/mount.go index 11702f720..1c3b2e987 100644 --- a/pkg/sentry/vfs/mount.go +++ b/pkg/sentry/vfs/mount.go @@ -38,16 +38,12 @@ import ( // Mount is analogous to Linux's struct mount. (gVisor does not distinguish // between struct mount and struct vfsmount.) type Mount struct { - // The lower 63 bits of refs are a reference count. The MSB of refs is set - // if the Mount has been eagerly unmounted, as by umount(2) without the - // MNT_DETACH flag. refs is accessed using atomic memory operations. - refs int64 - - // The lower 63 bits of writers is the number of calls to - // Mount.CheckBeginWrite() that have not yet been paired with a call to - // Mount.EndWrite(). The MSB of writers is set if MS_RDONLY is in effect. - // writers is accessed using atomic memory operations. - writers int64 + // vfs, fs, and root are immutable. References are held on fs and root. + // + // Invariant: root belongs to fs. + vfs *VirtualFilesystem + fs *Filesystem + root *Dentry // key is protected by VirtualFilesystem.mountMu and // VirtualFilesystem.mounts.seq, and may be nil. References are held on @@ -57,13 +53,29 @@ type Mount struct { // key.parent.fs. key mountKey - // fs, root, and ns are immutable. References are held on fs and root (but - // not ns). - // - // Invariant: root belongs to fs. - fs *Filesystem - root *Dentry - ns *MountNamespace + // ns is the namespace in which this Mount was mounted. ns is protected by + // VirtualFilesystem.mountMu. + ns *MountNamespace + + // The lower 63 bits of refs are a reference count. The MSB of refs is set + // if the Mount has been eagerly umounted, as by umount(2) without the + // MNT_DETACH flag. refs is accessed using atomic memory operations. + refs int64 + + // children is the set of all Mounts for which Mount.key.parent is this + // Mount. children is protected by VirtualFilesystem.mountMu. + children map[*Mount]struct{} + + // umounted is true if VFS.umountRecursiveLocked() has been called on this + // Mount. VirtualFilesystem does not hold a reference on Mounts for which + // umounted is true. umounted is protected by VirtualFilesystem.mountMu. + umounted bool + + // The lower 63 bits of writers is the number of calls to + // Mount.CheckBeginWrite() that have not yet been paired with a call to + // Mount.EndWrite(). The MSB of writers is set if MS_RDONLY is in effect. + // writers is accessed using atomic memory operations. + writers int64 } // A MountNamespace is a collection of Mounts. @@ -73,13 +85,16 @@ type Mount struct { // // MountNamespace is analogous to Linux's struct mnt_namespace. type MountNamespace struct { - refs int64 // accessed using atomic memory operations - // root is the MountNamespace's root mount. root is immutable. root *Mount - // mountpoints contains all Dentries which are mount points in this - // namespace. mountpoints is protected by VirtualFilesystem.mountMu. + // refs is the reference count. refs is accessed using atomic memory + // operations. + refs int64 + + // mountpoints maps all Dentries which are mount points in this namespace + // to the number of Mounts for which they are mount points. mountpoints is + // protected by VirtualFilesystem.mountMu. // // mountpoints is used to determine if a Dentry can be moved or removed // (which requires that the Dentry is not a mount point in the calling @@ -89,26 +104,27 @@ type MountNamespace struct { // MountNamespace; this is required to ensure that // VFS.PrepareDeleteDentry() and VFS.PrepareRemoveDentry() operate // correctly on unreferenced MountNamespaces. - mountpoints map[*Dentry]struct{} + mountpoints map[*Dentry]uint32 } // NewMountNamespace returns a new mount namespace with a root filesystem // configured by the given arguments. A reference is taken on the returned // MountNamespace. -func (vfs *VirtualFilesystem) NewMountNamespace(ctx context.Context, creds *auth.Credentials, source, fsTypeName string, opts *NewFilesystemOptions) (*MountNamespace, error) { +func (vfs *VirtualFilesystem) NewMountNamespace(ctx context.Context, creds *auth.Credentials, source, fsTypeName string, opts *GetFilesystemOptions) (*MountNamespace, error) { fsType := vfs.getFilesystemType(fsTypeName) if fsType == nil { return nil, syserror.ENODEV } - fs, root, err := fsType.NewFilesystem(ctx, creds, source, *opts) + fs, root, err := fsType.GetFilesystem(ctx, vfs, creds, source, *opts) if err != nil { return nil, err } mntns := &MountNamespace{ refs: 1, - mountpoints: make(map[*Dentry]struct{}), + mountpoints: make(map[*Dentry]uint32), } mntns.root = &Mount{ + vfs: vfs, fs: fs, root: root, ns: mntns, @@ -117,13 +133,13 @@ func (vfs *VirtualFilesystem) NewMountNamespace(ctx context.Context, creds *auth return mntns, nil } -// NewMount creates and mounts a new Filesystem. -func (vfs *VirtualFilesystem) NewMount(ctx context.Context, creds *auth.Credentials, source string, target *PathOperation, fsTypeName string, opts *NewFilesystemOptions) error { +// NewMount creates and mounts a Filesystem configured by the given arguments. +func (vfs *VirtualFilesystem) NewMount(ctx context.Context, creds *auth.Credentials, source string, target *PathOperation, fsTypeName string, opts *GetFilesystemOptions) error { fsType := vfs.getFilesystemType(fsTypeName) if fsType == nil { return syserror.ENODEV } - fs, root, err := fsType.NewFilesystem(ctx, creds, source, *opts) + fs, root, err := fsType.GetFilesystem(ctx, vfs, creds, source, *opts) if err != nil { return err } @@ -131,17 +147,19 @@ func (vfs *VirtualFilesystem) NewMount(ctx context.Context, creds *auth.Credenti // lock ordering. vd, err := vfs.GetDentryAt(ctx, creds, target, &GetDentryOptions{}) if err != nil { - root.decRef(fs) - fs.decRef() + root.DecRef() + fs.DecRef() return err } vfs.mountMu.Lock() + vd.dentry.mu.Lock() for { if vd.dentry.IsDisowned() { + vd.dentry.mu.Unlock() vfs.mountMu.Unlock() vd.DecRef() - root.decRef(fs) - fs.decRef() + root.DecRef() + fs.DecRef() return syserror.ENOENT } // vd might have been mounted over between vfs.GetDentryAt() and @@ -153,36 +171,210 @@ func (vfs *VirtualFilesystem) NewMount(ctx context.Context, creds *auth.Credenti if nextmnt == nil { break } - nextmnt.incRef() - nextmnt.root.incRef(nextmnt.fs) + // It's possible that nextmnt has been umounted but not disconnected, + // in which case vfs no longer holds a reference on it, and the last + // reference may be concurrently dropped even though we're holding + // vfs.mountMu. + if !nextmnt.tryIncMountedRef() { + break + } + // This can't fail since we're holding vfs.mountMu. + nextmnt.root.IncRef() + vd.dentry.mu.Unlock() vd.DecRef() vd = VirtualDentry{ mount: nextmnt, dentry: nextmnt.root, } + vd.dentry.mu.Lock() } // TODO: Linux requires that either both the mount point and the mount root // are directories, or neither are, and returns ENOTDIR if this is not the // case. mntns := vd.mount.ns mnt := &Mount{ + vfs: vfs, fs: fs, root: root, ns: mntns, refs: 1, } - mnt.storeKey(vd.mount, vd.dentry) + vfs.mounts.seq.BeginWrite() + vfs.connectLocked(mnt, vd, mntns) + vfs.mounts.seq.EndWrite() + vd.dentry.mu.Unlock() + vfs.mountMu.Unlock() + return nil +} + +type umountRecursiveOptions struct { + // If eager is true, ensure that future calls to Mount.tryIncMountedRef() + // on umounted mounts fail. + // + // eager is analogous to Linux's UMOUNT_SYNC. + eager bool + + // If disconnectHierarchy is true, Mounts that are umounted hierarchically + // should be disconnected from their parents. (Mounts whose parents are not + // umounted, which in most cases means the Mount passed to the initial call + // to umountRecursiveLocked, are unconditionally disconnected for + // consistency with Linux.) + // + // disconnectHierarchy is analogous to Linux's !UMOUNT_CONNECTED. + disconnectHierarchy bool +} + +// umountRecursiveLocked marks mnt and its descendants as umounted. It does not +// release mount or dentry references; instead, it appends VirtualDentries and +// Mounts on which references must be dropped to vdsToDecRef and mountsToDecRef +// respectively, and returns updated slices. (This is necessary because +// filesystem locks possibly taken by DentryImpl.DecRef() may precede +// vfs.mountMu in the lock order, and Mount.DecRef() may lock vfs.mountMu.) +// +// umountRecursiveLocked is analogous to Linux's fs/namespace.c:umount_tree(). +// +// Preconditions: vfs.mountMu must be locked. vfs.mounts.seq must be in a +// writer critical section. +func (vfs *VirtualFilesystem) umountRecursiveLocked(mnt *Mount, opts *umountRecursiveOptions, vdsToDecRef []VirtualDentry, mountsToDecRef []*Mount) ([]VirtualDentry, []*Mount) { + if !mnt.umounted { + mnt.umounted = true + mountsToDecRef = append(mountsToDecRef, mnt) + if parent := mnt.parent(); parent != nil && (opts.disconnectHierarchy || !parent.umounted) { + vdsToDecRef = append(vdsToDecRef, vfs.disconnectLocked(mnt)) + } + } + if opts.eager { + for { + refs := atomic.LoadInt64(&mnt.refs) + if refs < 0 { + break + } + if atomic.CompareAndSwapInt64(&mnt.refs, refs, refs|math.MinInt64) { + break + } + } + } + for child := range mnt.children { + vdsToDecRef, mountsToDecRef = vfs.umountRecursiveLocked(child, opts, vdsToDecRef, mountsToDecRef) + } + return vdsToDecRef, mountsToDecRef +} + +// connectLocked makes vd the mount parent/point for mnt. It consumes +// references held by vd. +// +// Preconditions: vfs.mountMu must be locked. vfs.mounts.seq must be in a +// writer critical section. d.mu must be locked. mnt.parent() == nil. +func (vfs *VirtualFilesystem) connectLocked(mnt *Mount, vd VirtualDentry, mntns *MountNamespace) { + mnt.storeKey(vd) + if vd.mount.children == nil { + vd.mount.children = make(map[*Mount]struct{}) + } + vd.mount.children[mnt] = struct{}{} atomic.AddUint32(&vd.dentry.mounts, 1) - mntns.mountpoints[vd.dentry] = struct{}{} + mntns.mountpoints[vd.dentry]++ + vfs.mounts.insertSeqed(mnt) vfsmpmounts, ok := vfs.mountpoints[vd.dentry] if !ok { vfsmpmounts = make(map[*Mount]struct{}) vfs.mountpoints[vd.dentry] = vfsmpmounts } vfsmpmounts[mnt] = struct{}{} - vfs.mounts.Insert(mnt) - vfs.mountMu.Unlock() - return nil +} + +// disconnectLocked makes vd have no mount parent/point and returns its old +// mount parent/point with a reference held. +// +// Preconditions: vfs.mountMu must be locked. vfs.mounts.seq must be in a +// writer critical section. mnt.parent() != nil. +func (vfs *VirtualFilesystem) disconnectLocked(mnt *Mount) VirtualDentry { + vd := mnt.loadKey() + mnt.storeKey(VirtualDentry{}) + delete(vd.mount.children, mnt) + atomic.AddUint32(&vd.dentry.mounts, math.MaxUint32) // -1 + mnt.ns.mountpoints[vd.dentry]-- + if mnt.ns.mountpoints[vd.dentry] == 0 { + delete(mnt.ns.mountpoints, vd.dentry) + } + vfs.mounts.removeSeqed(mnt) + vfsmpmounts := vfs.mountpoints[vd.dentry] + delete(vfsmpmounts, mnt) + if len(vfsmpmounts) == 0 { + delete(vfs.mountpoints, vd.dentry) + } + return vd +} + +// tryIncMountedRef increments mnt's reference count and returns true. If mnt's +// reference count is already zero, or has been eagerly umounted, +// tryIncMountedRef does nothing and returns false. +// +// tryIncMountedRef does not require that a reference is held on mnt. +func (mnt *Mount) tryIncMountedRef() bool { + for { + refs := atomic.LoadInt64(&mnt.refs) + if refs <= 0 { // refs < 0 => MSB set => eagerly unmounted + return false + } + if atomic.CompareAndSwapInt64(&mnt.refs, refs, refs+1) { + return true + } + } +} + +// IncRef increments mnt's reference count. +func (mnt *Mount) IncRef() { + // In general, negative values for mnt.refs are valid because the MSB is + // the eager-unmount bit. + atomic.AddInt64(&mnt.refs, 1) +} + +// DecRef decrements mnt's reference count. +func (mnt *Mount) DecRef() { + refs := atomic.AddInt64(&mnt.refs, -1) + if refs&^math.MinInt64 == 0 { // mask out MSB + var vd VirtualDentry + if mnt.parent() != nil { + mnt.vfs.mountMu.Lock() + mnt.vfs.mounts.seq.BeginWrite() + vd = mnt.vfs.disconnectLocked(mnt) + mnt.vfs.mounts.seq.EndWrite() + mnt.vfs.mountMu.Unlock() + } + mnt.root.DecRef() + mnt.fs.DecRef() + if vd.Ok() { + vd.DecRef() + } + } +} + +// IncRef increments mntns' reference count. +func (mntns *MountNamespace) IncRef() { + if atomic.AddInt64(&mntns.refs, 1) <= 1 { + panic("MountNamespace.IncRef() called without holding a reference") + } +} + +// DecRef decrements mntns' reference count. +func (mntns *MountNamespace) DecRef(vfs *VirtualFilesystem) { + if refs := atomic.AddInt64(&mntns.refs, -1); refs == 0 { + vfs.mountMu.Lock() + vfs.mounts.seq.BeginWrite() + vdsToDecRef, mountsToDecRef := vfs.umountRecursiveLocked(mntns.root, &umountRecursiveOptions{ + disconnectHierarchy: true, + }, nil, nil) + vfs.mounts.seq.EndWrite() + vfs.mountMu.Unlock() + for _, vd := range vdsToDecRef { + vd.DecRef() + } + for _, mnt := range mountsToDecRef { + mnt.DecRef() + } + } else if refs < 0 { + panic("MountNamespace.DecRef() called without holding a reference") + } } // getMountAt returns the last Mount in the stack mounted at (mnt, d). It takes @@ -223,7 +415,7 @@ retryFirst: // Raced with umount. continue } - mnt.decRef() + mnt.DecRef() mnt = next d = next.root } @@ -231,12 +423,12 @@ retryFirst: } // getMountpointAt returns the mount point for the stack of Mounts including -// mnt. It takes a reference on the returned Mount and Dentry. If no such mount +// mnt. It takes a reference on the returned VirtualDentry. If no such mount // point exists (i.e. mnt is a root mount), getMountpointAt returns (nil, nil). // // Preconditions: References are held on mnt and root. vfsroot is not (mnt, // mnt.root). -func (vfs *VirtualFilesystem) getMountpointAt(mnt *Mount, vfsroot VirtualDentry) (*Mount, *Dentry) { +func (vfs *VirtualFilesystem) getMountpointAt(mnt *Mount, vfsroot VirtualDentry) VirtualDentry { // The first mount is special-cased: // // - The caller must have already checked mnt against vfsroot. @@ -246,21 +438,26 @@ func (vfs *VirtualFilesystem) getMountpointAt(mnt *Mount, vfsroot VirtualDentry) // - We don't drop the caller's reference on mnt. retryFirst: epoch := vfs.mounts.seq.BeginRead() - parent, point := mnt.loadKey() + parent, point := mnt.parent(), mnt.point() if !vfs.mounts.seq.ReadOk(epoch) { goto retryFirst } if parent == nil { - return nil, nil + return VirtualDentry{} } if !parent.tryIncMountedRef() { // Raced with umount. goto retryFirst } - if !point.tryIncRef(parent.fs) { + if !point.TryIncRef() { // Since Mount holds a reference on Mount.key.point, this can only // happen due to a racing change to Mount.key. - parent.decRef() + parent.DecRef() + goto retryFirst + } + if !vfs.mounts.seq.ReadOk(epoch) { + point.DecRef() + parent.DecRef() goto retryFirst } mnt = parent @@ -274,7 +471,7 @@ retryFirst: } retryNotFirst: epoch := vfs.mounts.seq.BeginRead() - parent, point := mnt.loadKey() + parent, point := mnt.parent(), mnt.point() if !vfs.mounts.seq.ReadOk(epoch) { goto retryNotFirst } @@ -285,59 +482,23 @@ retryFirst: // Raced with umount. goto retryNotFirst } - if !point.tryIncRef(parent.fs) { + if !point.TryIncRef() { // Since Mount holds a reference on Mount.key.point, this can // only happen due to a racing change to Mount.key. - parent.decRef() + parent.DecRef() goto retryNotFirst } if !vfs.mounts.seq.ReadOk(epoch) { - point.decRef(parent.fs) - parent.decRef() + point.DecRef() + parent.DecRef() goto retryNotFirst } - d.decRef(mnt.fs) - mnt.decRef() + d.DecRef() + mnt.DecRef() mnt = parent d = point } - return mnt, d -} - -// tryIncMountedRef increments mnt's reference count and returns true. If mnt's -// reference count is already zero, or has been eagerly unmounted, -// tryIncMountedRef does nothing and returns false. -// -// tryIncMountedRef does not require that a reference is held on mnt. -func (mnt *Mount) tryIncMountedRef() bool { - for { - refs := atomic.LoadInt64(&mnt.refs) - if refs <= 0 { // refs < 0 => MSB set => eagerly unmounted - return false - } - if atomic.CompareAndSwapInt64(&mnt.refs, refs, refs+1) { - return true - } - } -} - -func (mnt *Mount) incRef() { - // In general, negative values for mnt.refs are valid because the MSB is - // the eager-unmount bit. - atomic.AddInt64(&mnt.refs, 1) -} - -func (mnt *Mount) decRef() { - refs := atomic.AddInt64(&mnt.refs, -1) - if refs&^math.MinInt64 == 0 { // mask out MSB - parent, point := mnt.loadKey() - if point != nil { - point.decRef(parent.fs) - parent.decRef() - } - mnt.root.decRef(mnt.fs) - mnt.fs.decRef() - } + return VirtualDentry{mnt, d} } // CheckBeginWrite increments the counter of in-progress write operations on @@ -360,7 +521,7 @@ func (mnt *Mount) EndWrite() { atomic.AddInt64(&mnt.writers, -1) } -// Preconditions: VirtualFilesystem.mountMu must be locked for writing. +// Preconditions: VirtualFilesystem.mountMu must be locked. func (mnt *Mount) setReadOnlyLocked(ro bool) error { if oldRO := atomic.LoadInt64(&mnt.writers) < 0; oldRO == ro { return nil @@ -383,22 +544,6 @@ func (mnt *Mount) Filesystem() *Filesystem { return mnt.fs } -// IncRef increments mntns' reference count. -func (mntns *MountNamespace) IncRef() { - if atomic.AddInt64(&mntns.refs, 1) <= 1 { - panic("MountNamespace.IncRef() called without holding a reference") - } -} - -// DecRef decrements mntns' reference count. -func (mntns *MountNamespace) DecRef() { - if refs := atomic.AddInt64(&mntns.refs, 0); refs == 0 { - // TODO: unmount mntns.root - } else if refs < 0 { - panic("MountNamespace.DecRef() called without holding a reference") - } -} - // Root returns mntns' root. A reference is taken on the returned // VirtualDentry. func (mntns *MountNamespace) Root() VirtualDentry { diff --git a/pkg/sentry/vfs/mount_test.go b/pkg/sentry/vfs/mount_test.go index f394d7483..adff0b94b 100644 --- a/pkg/sentry/vfs/mount_test.go +++ b/pkg/sentry/vfs/mount_test.go @@ -37,7 +37,7 @@ func TestMountTableInsertLookup(t *testing.T) { mt.Init() mount := &Mount{} - mount.storeKey(&Mount{}, &Dentry{}) + mount.storeKey(VirtualDentry{&Mount{}, &Dentry{}}) mt.Insert(mount) if m := mt.Lookup(mount.parent(), mount.point()); m != mount { @@ -78,18 +78,10 @@ const enableComparativeBenchmarks = false func newBenchMount() *Mount { mount := &Mount{} - mount.storeKey(&Mount{}, &Dentry{}) + mount.storeKey(VirtualDentry{&Mount{}, &Dentry{}}) return mount } -func vdkey(mnt *Mount) VirtualDentry { - parent, point := mnt.loadKey() - return VirtualDentry{ - mount: parent, - dentry: point, - } -} - func BenchmarkMountTableParallelLookup(b *testing.B) { for numG, maxG := 1, runtime.GOMAXPROCS(0); numG >= 0 && numG <= maxG; numG *= 2 { for _, numMounts := range benchNumMounts { @@ -101,7 +93,7 @@ func BenchmarkMountTableParallelLookup(b *testing.B) { for i := 0; i < numMounts; i++ { mount := newBenchMount() mt.Insert(mount) - keys = append(keys, vdkey(mount)) + keys = append(keys, mount.loadKey()) } var ready sync.WaitGroup @@ -153,7 +145,7 @@ func BenchmarkMountMapParallelLookup(b *testing.B) { keys := make([]VirtualDentry, 0, numMounts) for i := 0; i < numMounts; i++ { mount := newBenchMount() - key := vdkey(mount) + key := mount.loadKey() ms[key] = mount keys = append(keys, key) } @@ -208,7 +200,7 @@ func BenchmarkMountSyncMapParallelLookup(b *testing.B) { keys := make([]VirtualDentry, 0, numMounts) for i := 0; i < numMounts; i++ { mount := newBenchMount() - key := vdkey(mount) + key := mount.loadKey() ms.Store(key, mount) keys = append(keys, key) } @@ -290,7 +282,7 @@ func BenchmarkMountMapNegativeLookup(b *testing.B) { ms := make(map[VirtualDentry]*Mount) for i := 0; i < numMounts; i++ { mount := newBenchMount() - ms[vdkey(mount)] = mount + ms[mount.loadKey()] = mount } negkeys := make([]VirtualDentry, 0, numMounts) for i := 0; i < numMounts; i++ { @@ -325,7 +317,7 @@ func BenchmarkMountSyncMapNegativeLookup(b *testing.B) { var ms sync.Map for i := 0; i < numMounts; i++ { mount := newBenchMount() - ms.Store(vdkey(mount), mount) + ms.Store(mount.loadKey(), mount) } negkeys := make([]VirtualDentry, 0, numMounts) for i := 0; i < numMounts; i++ { @@ -379,7 +371,7 @@ func BenchmarkMountMapInsert(b *testing.B) { b.ResetTimer() for i := range mounts { mount := mounts[i] - ms[vdkey(mount)] = mount + ms[mount.loadKey()] = mount } } @@ -399,7 +391,7 @@ func BenchmarkMountSyncMapInsert(b *testing.B) { b.ResetTimer() for i := range mounts { mount := mounts[i] - ms.Store(vdkey(mount), mount) + ms.Store(mount.loadKey(), mount) } } @@ -432,13 +424,13 @@ func BenchmarkMountMapRemove(b *testing.B) { ms := make(map[VirtualDentry]*Mount) for i := range mounts { mount := mounts[i] - ms[vdkey(mount)] = mount + ms[mount.loadKey()] = mount } b.ResetTimer() for i := range mounts { mount := mounts[i] - delete(ms, vdkey(mount)) + delete(ms, mount.loadKey()) } } @@ -454,12 +446,12 @@ func BenchmarkMountSyncMapRemove(b *testing.B) { var ms sync.Map for i := range mounts { mount := mounts[i] - ms.Store(vdkey(mount), mount) + ms.Store(mount.loadKey(), mount) } b.ResetTimer() for i := range mounts { mount := mounts[i] - ms.Delete(vdkey(mount)) + ms.Delete(mount.loadKey()) } } diff --git a/pkg/sentry/vfs/mount_unsafe.go b/pkg/sentry/vfs/mount_unsafe.go index c98b42f91..ab13fa461 100644 --- a/pkg/sentry/vfs/mount_unsafe.go +++ b/pkg/sentry/vfs/mount_unsafe.go @@ -38,16 +38,6 @@ type mountKey struct { point unsafe.Pointer // *Dentry } -// Invariant: mnt.key's fields are nil. parent and point are non-nil. -func (mnt *Mount) storeKey(parent *Mount, point *Dentry) { - atomic.StorePointer(&mnt.key.parent, unsafe.Pointer(parent)) - atomic.StorePointer(&mnt.key.point, unsafe.Pointer(point)) -} - -func (mnt *Mount) loadKey() (*Mount, *Dentry) { - return (*Mount)(atomic.LoadPointer(&mnt.key.parent)), (*Dentry)(atomic.LoadPointer(&mnt.key.point)) -} - func (mnt *Mount) parent() *Mount { return (*Mount)(atomic.LoadPointer(&mnt.key.parent)) } @@ -56,6 +46,19 @@ func (mnt *Mount) point() *Dentry { return (*Dentry)(atomic.LoadPointer(&mnt.key.point)) } +func (mnt *Mount) loadKey() VirtualDentry { + return VirtualDentry{ + mount: mnt.parent(), + dentry: mnt.point(), + } +} + +// Invariant: mnt.key.parent == nil. vd.Ok(). +func (mnt *Mount) storeKey(vd VirtualDentry) { + atomic.StorePointer(&mnt.key.parent, unsafe.Pointer(vd.mount)) + atomic.StorePointer(&mnt.key.point, unsafe.Pointer(vd.dentry)) +} + // mountTable maps (mount parent, mount point) pairs to mounts. It supports // efficient concurrent lookup, even in the presence of concurrent mutators // (provided mutation is sufficiently uncommon). @@ -201,9 +204,19 @@ loop: // Insert inserts the given mount into mt. // -// Preconditions: There are no concurrent mutators of mt. mt must not already -// contain a Mount with the same mount point and parent. +// Preconditions: mt must not already contain a Mount with the same mount point +// and parent. func (mt *mountTable) Insert(mount *Mount) { + mt.seq.BeginWrite() + mt.insertSeqed(mount) + mt.seq.EndWrite() +} + +// insertSeqed inserts the given mount into mt. +// +// Preconditions: mt.seq must be in a writer critical section. mt must not +// already contain a Mount with the same mount point and parent. +func (mt *mountTable) insertSeqed(mount *Mount) { hash := memhash(unsafe.Pointer(&mount.key), uintptr(mt.seed), mountKeyBytes) // We're under the maximum load factor if: @@ -215,10 +228,8 @@ func (mt *mountTable) Insert(mount *Mount) { tcap := uintptr(1) << order if ((tlen + 1) * mtMaxLoadDen) <= (uint64(mtMaxLoadNum) << order) { // Atomically insert the new element into the table. - mt.seq.BeginWrite() atomic.AddUint64(&mt.size, mtSizeLenOne) mtInsertLocked(mt.slots, tcap, unsafe.Pointer(mount), hash) - mt.seq.EndWrite() return } @@ -241,8 +252,6 @@ func (mt *mountTable) Insert(mount *Mount) { for { oldSlot := (*mountSlot)(oldCur) if oldSlot.value != nil { - // Don't need to lock mt.seq yet since newSlots isn't visible - // to readers. mtInsertLocked(newSlots, newCap, oldSlot.value, oldSlot.hash) } if oldCur == oldLast { @@ -252,11 +261,9 @@ func (mt *mountTable) Insert(mount *Mount) { } // Insert the new element into the new table. mtInsertLocked(newSlots, newCap, unsafe.Pointer(mount), hash) - // Atomically switch to the new table. - mt.seq.BeginWrite() + // Switch to the new table. atomic.AddUint64(&mt.size, mtSizeLenOne|mtSizeOrderOne) atomic.StorePointer(&mt.slots, newSlots) - mt.seq.EndWrite() } // Preconditions: There are no concurrent mutators of the table (slots, cap). @@ -294,9 +301,18 @@ func mtInsertLocked(slots unsafe.Pointer, cap uintptr, value unsafe.Pointer, has // Remove removes the given mount from mt. // -// Preconditions: There are no concurrent mutators of mt. mt must contain -// mount. +// Preconditions: mt must contain mount. func (mt *mountTable) Remove(mount *Mount) { + mt.seq.BeginWrite() + mt.removeSeqed(mount) + mt.seq.EndWrite() +} + +// removeSeqed removes the given mount from mt. +// +// Preconditions: mt.seq must be in a writer critical section. mt must contain +// mount. +func (mt *mountTable) removeSeqed(mount *Mount) { hash := memhash(unsafe.Pointer(&mount.key), uintptr(mt.seed), mountKeyBytes) tcap := uintptr(1) << (mt.size & mtSizeOrderMask) mask := tcap - 1 @@ -311,7 +327,6 @@ func (mt *mountTable) Remove(mount *Mount) { // backward until we either find an empty slot, or an element that // is already in its first-probed slot. (This is backward shift // deletion.) - mt.seq.BeginWrite() for { nextOff := (off + mountSlotBytes) & offmask nextSlot := (*mountSlot)(unsafe.Pointer(uintptr(slots) + nextOff)) @@ -330,7 +345,6 @@ func (mt *mountTable) Remove(mount *Mount) { } atomic.StorePointer(&slot.value, nil) atomic.AddUint64(&mt.size, mtSizeLenNegOne) - mt.seq.EndWrite() return } if checkInvariants && slotValue == nil { diff --git a/pkg/sentry/vfs/resolving_path.go b/pkg/sentry/vfs/resolving_path.go index 8d05c8583..621f5a6f8 100644 --- a/pkg/sentry/vfs/resolving_path.go +++ b/pkg/sentry/vfs/resolving_path.go @@ -149,20 +149,20 @@ func (vfs *VirtualFilesystem) putResolvingPath(rp *ResolvingPath) { func (rp *ResolvingPath) decRefStartAndMount() { if rp.flags&rpflagsHaveStartRef != 0 { - rp.start.decRef(rp.mount.fs) + rp.start.DecRef() } if rp.flags&rpflagsHaveMountRef != 0 { - rp.mount.decRef() + rp.mount.DecRef() } } func (rp *ResolvingPath) releaseErrorState() { if rp.nextStart != nil { - rp.nextStart.decRef(rp.nextMount.fs) + rp.nextStart.DecRef() rp.nextStart = nil } if rp.nextMount != nil { - rp.nextMount.decRef() + rp.nextMount.DecRef() rp.nextMount = nil } } @@ -269,11 +269,11 @@ func (rp *ResolvingPath) ResolveParent(d *Dentry) (*Dentry, error) { parent = d } else if d == rp.mount.root { // At mount root ... - mnt, mntpt := rp.vfs.getMountpointAt(rp.mount, rp.root) - if mnt != nil { + vd := rp.vfs.getMountpointAt(rp.mount, rp.root) + if vd.Ok() { // ... of non-root mount. - rp.nextMount = mnt - rp.nextStart = mntpt + rp.nextMount = vd.mount + rp.nextStart = vd.dentry return nil, resolveMountRootError{} } // ... of root mount. diff --git a/pkg/sentry/vfs/syscalls.go b/pkg/sentry/vfs/syscalls.go index abde0feaa..436151afa 100644 --- a/pkg/sentry/vfs/syscalls.go +++ b/pkg/sentry/vfs/syscalls.go @@ -63,7 +63,7 @@ func (vfs *VirtualFilesystem) GetDentryAt(ctx context.Context, creds *auth.Crede mount: rp.mount, dentry: d, } - rp.mount.incRef() + rp.mount.IncRef() vfs.putResolvingPath(rp) return vd, nil } @@ -230,6 +230,8 @@ func (fd *FileDescription) SetStatusFlags(ctx context.Context, flags uint32) err // // - VFS.SymlinkAt() // +// - VFS.UmountAt() +// // - VFS.UnlinkAt() // // - FileDescription.(almost everything) diff --git a/pkg/sentry/vfs/testutil.go b/pkg/sentry/vfs/testutil.go index 70b192ece..593144cb7 100644 --- a/pkg/sentry/vfs/testutil.go +++ b/pkg/sentry/vfs/testutil.go @@ -33,10 +33,10 @@ type FDTestFilesystem struct { vfsfs Filesystem } -// NewFilesystem implements FilesystemType.NewFilesystem. -func (fstype FDTestFilesystemType) NewFilesystem(ctx context.Context, creds *auth.Credentials, source string, opts NewFilesystemOptions) (*Filesystem, *Dentry, error) { +// GetFilesystem implements FilesystemType.GetFilesystem. +func (fstype FDTestFilesystemType) GetFilesystem(ctx context.Context, vfsObj *VirtualFilesystem, creds *auth.Credentials, source string, opts GetFilesystemOptions) (*Filesystem, *Dentry, error) { var fs FDTestFilesystem - fs.vfsfs.Init(&fs) + fs.vfsfs.Init(vfsObj, &fs) return &fs.vfsfs, fs.NewDentry(), nil } @@ -126,14 +126,14 @@ func (fs *FDTestFilesystem) NewDentry() *Dentry { } // IncRef implements DentryImpl.IncRef. -func (d *fdTestDentry) IncRef(vfsfs *Filesystem) { +func (d *fdTestDentry) IncRef() { } // TryIncRef implements DentryImpl.TryIncRef. -func (d *fdTestDentry) TryIncRef(vfsfs *Filesystem) bool { +func (d *fdTestDentry) TryIncRef() bool { return true } // DecRef implements DentryImpl.DecRef. -func (d *fdTestDentry) DecRef(vfsfs *Filesystem) { +func (d *fdTestDentry) DecRef() { } diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go index 4a8a69540..f0cd3ffe5 100644 --- a/pkg/sentry/vfs/vfs.go +++ b/pkg/sentry/vfs/vfs.go @@ -16,9 +16,14 @@ // // Lock order: // -// Filesystem implementation locks +// FilesystemImpl/FileDescriptionImpl locks // VirtualFilesystem.mountMu +// Dentry.mu +// Locks acquired by FilesystemImpls between Prepare{Delete,Rename}Dentry and Commit{Delete,Rename*}Dentry // VirtualFilesystem.fsTypesMu +// +// Locking Dentry.mu in multiple Dentries requires holding +// VirtualFilesystem.mountMu. package vfs import ( @@ -33,7 +38,7 @@ type VirtualFilesystem struct { // mountMu serializes mount mutations. // // mountMu is analogous to Linux's namespace_sem. - mountMu sync.RWMutex + mountMu sync.Mutex // mounts maps (mount parent, mount point) pairs to mounts. (Since mounts // are uniquely namespaced, including mount parent in the key correctly @@ -52,7 +57,7 @@ type VirtualFilesystem struct { // mountpoints maps mount points to mounts at those points in all // namespaces. mountpoints is protected by mountMu. // - // mountpoints is used to find mounts that must be unmounted due to + // mountpoints is used to find mounts that must be umounted due to // removal of a mount point Dentry from another mount namespace. ("A file // or directory that is a mount point in one namespace that is not a mount // point in another namespace, may be renamed, unlinked, or removed @@ -111,15 +116,15 @@ func (vd VirtualDentry) Ok() bool { // IncRef increments the reference counts on the Mount and Dentry represented // by vd. func (vd VirtualDentry) IncRef() { - vd.mount.incRef() - vd.dentry.incRef(vd.mount.fs) + vd.mount.IncRef() + vd.dentry.IncRef() } // DecRef decrements the reference counts on the Mount and Dentry represented // by vd. func (vd VirtualDentry) DecRef() { - vd.dentry.decRef(vd.mount.fs) - vd.mount.decRef() + vd.dentry.DecRef() + vd.mount.DecRef() } // Mount returns the Mount associated with vd. It does not take a reference on |