summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl/kernfs
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fsimpl/kernfs')
-rw-r--r--pkg/sentry/fsimpl/kernfs/BUILD37
-rw-r--r--pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go4
-rw-r--r--pkg/sentry/fsimpl/kernfs/fd_impl_util.go14
-rw-r--r--pkg/sentry/fsimpl/kernfs/filesystem.go5
-rw-r--r--pkg/sentry/fsimpl/kernfs/inode_impl_util.go96
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs.go205
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs_test.go62
-rw-r--r--pkg/sentry/fsimpl/kernfs/symlink.go8
-rw-r--r--pkg/sentry/fsimpl/kernfs/synthetic_directory.go11
9 files changed, 316 insertions, 126 deletions
diff --git a/pkg/sentry/fsimpl/kernfs/BUILD b/pkg/sentry/fsimpl/kernfs/BUILD
index 858cc24ce..aaad67ab8 100644
--- a/pkg/sentry/fsimpl/kernfs/BUILD
+++ b/pkg/sentry/fsimpl/kernfs/BUILD
@@ -4,6 +4,18 @@ load("//tools/go_generics:defs.bzl", "go_template_instance")
licenses(["notice"])
go_template_instance(
+ name = "dentry_list",
+ out = "dentry_list.go",
+ package = "kernfs",
+ prefix = "dentry",
+ template = "//pkg/ilist:generic_list",
+ types = {
+ "Element": "*Dentry",
+ "Linker": "*Dentry",
+ },
+)
+
+go_template_instance(
name = "fstree",
out = "fstree.go",
package = "kernfs",
@@ -27,22 +39,11 @@ go_template_instance(
)
go_template_instance(
- name = "dentry_refs",
- out = "dentry_refs.go",
- package = "kernfs",
- prefix = "Dentry",
- template = "//pkg/refs_vfs2:refs_template",
- types = {
- "T": "Dentry",
- },
-)
-
-go_template_instance(
name = "static_directory_refs",
out = "static_directory_refs.go",
package = "kernfs",
prefix = "StaticDirectory",
- template = "//pkg/refs_vfs2:refs_template",
+ template = "//pkg/refsvfs2:refs_template",
types = {
"T": "StaticDirectory",
},
@@ -53,7 +54,7 @@ go_template_instance(
out = "dir_refs.go",
package = "kernfs_test",
prefix = "dir",
- template = "//pkg/refs_vfs2:refs_template",
+ template = "//pkg/refsvfs2:refs_template",
types = {
"T": "dir",
},
@@ -64,7 +65,7 @@ go_template_instance(
out = "readonly_dir_refs.go",
package = "kernfs_test",
prefix = "readonlyDir",
- template = "//pkg/refs_vfs2:refs_template",
+ template = "//pkg/refsvfs2:refs_template",
types = {
"T": "readonlyDir",
},
@@ -75,7 +76,7 @@ go_template_instance(
out = "synthetic_directory_refs.go",
package = "kernfs",
prefix = "syntheticDirectory",
- template = "//pkg/refs_vfs2:refs_template",
+ template = "//pkg/refsvfs2:refs_template",
types = {
"T": "syntheticDirectory",
},
@@ -84,7 +85,7 @@ go_template_instance(
go_library(
name = "kernfs",
srcs = [
- "dentry_refs.go",
+ "dentry_list.go",
"dynamic_bytes_file.go",
"fd_impl_util.go",
"filesystem.go",
@@ -104,8 +105,11 @@ go_library(
"//pkg/fspath",
"//pkg/log",
"//pkg/refs",
+ "//pkg/refsvfs2",
+ "//pkg/sentry/fs/fsutil",
"//pkg/sentry/fs/lock",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/time",
"//pkg/sentry/memmap",
"//pkg/sentry/socket/unix/transport",
"//pkg/sentry/vfs",
@@ -129,6 +133,7 @@ go_test(
"//pkg/context",
"//pkg/log",
"//pkg/refs",
+ "//pkg/refsvfs2",
"//pkg/sentry/contexttest",
"//pkg/sentry/fsimpl/testutil",
"//pkg/sentry/kernel/auth",
diff --git a/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go b/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
index b929118b1..485504995 100644
--- a/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
+++ b/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
@@ -47,11 +47,11 @@ type DynamicBytesFile struct {
var _ Inode = (*DynamicBytesFile)(nil)
// Init initializes a dynamic bytes file.
-func (f *DynamicBytesFile) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, data vfs.DynamicBytesSource, perm linux.FileMode) {
+func (f *DynamicBytesFile) Init(ctx context.Context, 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, devMajor, devMinor, ino, linux.ModeRegular|perm)
+ f.InodeAttrs.Init(ctx, creds, devMajor, devMinor, ino, linux.ModeRegular|perm)
f.data = data
}
diff --git a/pkg/sentry/fsimpl/kernfs/fd_impl_util.go b/pkg/sentry/fsimpl/kernfs/fd_impl_util.go
index abf1905d6..f8dae22f8 100644
--- a/pkg/sentry/fsimpl/kernfs/fd_impl_util.go
+++ b/pkg/sentry/fsimpl/kernfs/fd_impl_util.go
@@ -145,8 +145,12 @@ func (fd *GenericDirectoryFD) filesystem() *vfs.Filesystem {
return fd.vfsfd.VirtualDentry().Mount().Filesystem()
}
+func (fd *GenericDirectoryFD) dentry() *Dentry {
+ return fd.vfsfd.Dentry().Impl().(*Dentry)
+}
+
func (fd *GenericDirectoryFD) inode() Inode {
- return fd.vfsfd.VirtualDentry().Dentry().Impl().(*Dentry).inode
+ return fd.dentry().inode
}
// IterDirents implements vfs.FileDescriptionImpl.IterDirents. IterDirents holds
@@ -176,8 +180,7 @@ func (fd *GenericDirectoryFD) IterDirents(ctx context.Context, cb vfs.IterDirent
// Handle "..".
if fd.off == 1 {
- vfsd := fd.vfsfd.VirtualDentry().Dentry()
- parentInode := genericParentOrSelf(vfsd.Impl().(*Dentry)).inode
+ parentInode := genericParentOrSelf(fd.dentry()).inode
stat, err := parentInode.Stat(ctx, fd.filesystem(), opts)
if err != nil {
return err
@@ -219,7 +222,7 @@ func (fd *GenericDirectoryFD) IterDirents(ctx context.Context, cb vfs.IterDirent
var err error
relOffset := fd.off - int64(len(fd.children.set)) - 2
- fd.off, err = fd.inode().IterDirents(ctx, cb, fd.off, relOffset)
+ fd.off, err = fd.inode().IterDirents(ctx, fd.vfsfd.Mount(), cb, fd.off, relOffset)
return err
}
@@ -265,8 +268,7 @@ func (fd *GenericDirectoryFD) Stat(ctx context.Context, opts vfs.StatOptions) (l
// SetStat implements vfs.FileDescriptionImpl.SetStat.
func (fd *GenericDirectoryFD) SetStat(ctx context.Context, opts vfs.SetStatOptions) error {
creds := auth.CredentialsFromContext(ctx)
- inode := fd.vfsfd.VirtualDentry().Dentry().Impl().(*Dentry).inode
- return inode.SetStat(ctx, fd.filesystem(), creds, opts)
+ return fd.inode().SetStat(ctx, fd.filesystem(), creds, opts)
}
// Allocate implements vfs.FileDescriptionImpl.Allocate.
diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go
index 6426a55f6..399895f3e 100644
--- a/pkg/sentry/fsimpl/kernfs/filesystem.go
+++ b/pkg/sentry/fsimpl/kernfs/filesystem.go
@@ -373,7 +373,7 @@ func (fs *Filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
if !opts.ForSyntheticMountpoint || err == syserror.EEXIST {
return err
}
- childI = newSyntheticDirectory(rp.Credentials(), opts.Mode)
+ childI = newSyntheticDirectory(ctx, rp.Credentials(), opts.Mode)
}
var child Dentry
child.Init(fs, childI)
@@ -517,9 +517,6 @@ afterTrailingSymlink:
}
var child Dentry
child.Init(fs, childI)
- // FIXME(gvisor.dev/issue/1193): Race between checking existence with
- // fs.stepExistingLocked and parent.insertChild. If possible, we should hold
- // dirMu from one to the other.
parent.insertChild(pc, &child)
// Open may block so we need to unlock fs.mu. IncRef child to prevent
// its destruction while fs.mu is unlocked.
diff --git a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
index 122b10591..d9d76758a 100644
--- a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
+++ b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
@@ -21,9 +21,11 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
)
// InodeNoopRefCount partially implements the Inode interface, specifically the
@@ -143,7 +145,7 @@ func (InodeNotDirectory) Lookup(ctx context.Context, name string) (Inode, error)
}
// IterDirents implements Inode.IterDirents.
-func (InodeNotDirectory) IterDirents(ctx context.Context, callback vfs.IterDirentsCallback, offset, relOffset int64) (newOffset int64, err error) {
+func (InodeNotDirectory) IterDirents(ctx context.Context, mnt *vfs.Mount, callback vfs.IterDirentsCallback, offset, relOffset int64) (newOffset int64, err error) {
panic("IterDirents called on non-directory inode")
}
@@ -172,17 +174,23 @@ func (InodeNotSymlink) Getlink(context.Context, *vfs.Mount) (vfs.VirtualDentry,
//
// +stateify savable
type InodeAttrs struct {
- devMajor uint32
- devMinor uint32
- ino uint64
- mode uint32
- uid uint32
- gid uint32
- nlink uint32
+ devMajor uint32
+ devMinor uint32
+ ino uint64
+ mode uint32
+ uid uint32
+ gid uint32
+ nlink uint32
+ blockSize uint32
+
+ // Timestamps, all nsecs from the Unix epoch.
+ atime int64
+ mtime int64
+ ctime int64
}
// Init initializes this InodeAttrs.
-func (a *InodeAttrs) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, mode linux.FileMode) {
+func (a *InodeAttrs) Init(ctx context.Context, 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))
}
@@ -198,6 +206,11 @@ func (a *InodeAttrs) Init(creds *auth.Credentials, devMajor, devMinor uint32, in
atomic.StoreUint32(&a.uid, uint32(creds.EffectiveKUID))
atomic.StoreUint32(&a.gid, uint32(creds.EffectiveKGID))
atomic.StoreUint32(&a.nlink, nlink)
+ atomic.StoreUint32(&a.blockSize, usermem.PageSize)
+ now := ktime.NowFromContext(ctx).Nanoseconds()
+ atomic.StoreInt64(&a.atime, now)
+ atomic.StoreInt64(&a.mtime, now)
+ atomic.StoreInt64(&a.ctime, now)
}
// DevMajor returns the device major number.
@@ -220,12 +233,33 @@ func (a *InodeAttrs) Mode() linux.FileMode {
return linux.FileMode(atomic.LoadUint32(&a.mode))
}
+// TouchAtime updates a.atime to the current time.
+func (a *InodeAttrs) TouchAtime(ctx context.Context, mnt *vfs.Mount) {
+ if mnt.Flags.NoATime || mnt.ReadOnly() {
+ return
+ }
+ if err := mnt.CheckBeginWrite(); err != nil {
+ return
+ }
+ atomic.StoreInt64(&a.atime, ktime.NowFromContext(ctx).Nanoseconds())
+ mnt.EndWrite()
+}
+
+// TouchCMtime updates a.{c/m}time to the current time. The caller should
+// synchronize calls to this so that ctime and mtime are updated to the same
+// value.
+func (a *InodeAttrs) TouchCMtime(ctx context.Context) {
+ now := ktime.NowFromContext(ctx).Nanoseconds()
+ atomic.StoreInt64(&a.mtime, now)
+ atomic.StoreInt64(&a.ctime, now)
+}
+
// Stat partially implements Inode.Stat. Note that this function doesn't provide
// all the stat fields, and the embedder should consider extending the result
// with filesystem-specific fields.
func (a *InodeAttrs) Stat(context.Context, *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.Mask = linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID | linux.STATX_INO | linux.STATX_NLINK | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME
stat.DevMajor = a.devMajor
stat.DevMinor = a.devMinor
stat.Ino = atomic.LoadUint64(&a.ino)
@@ -233,21 +267,15 @@ func (a *InodeAttrs) Stat(context.Context, *vfs.Filesystem, vfs.StatOptions) (li
stat.UID = atomic.LoadUint32(&a.uid)
stat.GID = atomic.LoadUint32(&a.gid)
stat.Nlink = atomic.LoadUint32(&a.nlink)
-
- // TODO(gvisor.dev/issue/1193): Implement other stat fields like timestamps.
-
+ stat.Blksize = atomic.LoadUint32(&a.blockSize)
+ stat.Atime = linux.NsecToStatxTimestamp(atomic.LoadInt64(&a.atime))
+ stat.Mtime = linux.NsecToStatxTimestamp(atomic.LoadInt64(&a.mtime))
+ stat.Ctime = linux.NsecToStatxTimestamp(atomic.LoadInt64(&a.ctime))
return stat, nil
}
// SetStat implements Inode.SetStat.
func (a *InodeAttrs) SetStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error {
- return a.SetInodeStat(ctx, fs, creds, opts)
-}
-
-// SetInodeStat sets the corresponding attributes from opts to InodeAttrs.
-// This function can be used by other kernfs-based filesystem implementation to
-// sets the unexported attributes into InodeAttrs.
-func (a *InodeAttrs) SetInodeStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error {
if opts.Stat.Mask == 0 {
return nil
}
@@ -256,9 +284,7 @@ func (a *InodeAttrs) SetInodeStat(ctx context.Context, fs *vfs.Filesystem, creds
// inode numbers are immutable after node creation. Setting the size is often
// allowed by kernfs files but does not do anything. If some other behavior is
// needed, the embedder should consider extending SetStat.
- //
- // TODO(gvisor.dev/issue/1193): Implement other stat fields like timestamps.
- if opts.Stat.Mask&^(linux.STATX_MODE|linux.STATX_UID|linux.STATX_GID|linux.STATX_SIZE) != 0 {
+ if opts.Stat.Mask&^(linux.STATX_MODE|linux.STATX_UID|linux.STATX_GID|linux.STATX_ATIME|linux.STATX_MTIME|linux.STATX_SIZE) != 0 {
return syserror.EPERM
}
if opts.Stat.Mask&linux.STATX_SIZE != 0 && a.Mode().IsDir() {
@@ -286,6 +312,20 @@ func (a *InodeAttrs) SetInodeStat(ctx context.Context, fs *vfs.Filesystem, creds
atomic.StoreUint32(&a.gid, stat.GID)
}
+ now := ktime.NowFromContext(ctx).Nanoseconds()
+ if stat.Mask&linux.STATX_ATIME != 0 {
+ if stat.Atime.Nsec == linux.UTIME_NOW {
+ stat.Atime = linux.NsecToStatxTimestamp(now)
+ }
+ atomic.StoreInt64(&a.atime, stat.Atime.ToNsec())
+ }
+ if stat.Mask&linux.STATX_MTIME != 0 {
+ if stat.Mtime.Nsec == linux.UTIME_NOW {
+ stat.Mtime = linux.NsecToStatxTimestamp(now)
+ }
+ atomic.StoreInt64(&a.mtime, stat.Mtime.ToNsec())
+ }
+
return nil
}
@@ -421,7 +461,7 @@ func (o *OrderedChildren) Lookup(ctx context.Context, name string) (Inode, error
}
// IterDirents implements Inode.IterDirents.
-func (o *OrderedChildren) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback, offset, relOffset int64) (newOffset int64, err error) {
+func (o *OrderedChildren) IterDirents(ctx context.Context, mnt *vfs.Mount, cb vfs.IterDirentsCallback, offset, relOffset int64) (newOffset int64, err error) {
// All entries from OrderedChildren have already been handled in
// GenericDirectoryFD.IterDirents.
return offset, nil
@@ -619,9 +659,9 @@ type StaticDirectory struct {
var _ Inode = (*StaticDirectory)(nil)
// NewStaticDir creates a new static directory and returns its dentry.
-func NewStaticDir(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode, children map[string]Inode, fdOpts GenericDirectoryFDOptions) Inode {
+func NewStaticDir(ctx context.Context, creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode, children map[string]Inode, fdOpts GenericDirectoryFDOptions) Inode {
inode := &StaticDirectory{}
- inode.Init(creds, devMajor, devMinor, ino, perm, fdOpts)
+ inode.Init(ctx, creds, devMajor, devMinor, ino, perm, fdOpts)
inode.EnableLeakCheck()
inode.OrderedChildren.Init(OrderedChildrenOptions{})
@@ -632,12 +672,12 @@ func NewStaticDir(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64
}
// Init initializes StaticDirectory.
-func (s *StaticDirectory) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode, fdOpts GenericDirectoryFDOptions) {
+func (s *StaticDirectory) Init(ctx context.Context, creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode, fdOpts GenericDirectoryFDOptions) {
if perm&^linux.PermissionsMask != 0 {
panic(fmt.Sprintf("Only permission mask must be set: %x", perm&linux.PermissionsMask))
}
s.fdOpts = fdOpts
- s.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.ModeDirectory|perm)
+ s.InodeAttrs.Init(ctx, creds, devMajor, devMinor, ino, linux.ModeDirectory|perm)
}
// Open implements Inode.Open.
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go
index 606081e68..5c5e09ac5 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs.go
@@ -107,6 +107,17 @@ type Filesystem struct {
// nextInoMinusOne is used to to allocate inode numbers on this
// filesystem. Must be accessed by atomic operations.
nextInoMinusOne uint64
+
+ // cachedDentries contains all dentries with 0 references. (Due to race
+ // conditions, it may also contain dentries with non-zero references.)
+ // cachedDentriesLen is the number of dentries in cachedDentries. These
+ // fields are protected by mu.
+ cachedDentries dentryList
+ cachedDentriesLen uint64
+
+ // MaxCachedDentries is the maximum size of cachedDentries. If not set,
+ // defaults to 0 and kernfs does not cache any dentries. This is immutable.
+ MaxCachedDentries uint64
}
// deferDecRef defers dropping a dentry ref until the next call to
@@ -165,7 +176,12 @@ const (
// +stateify savable
type Dentry struct {
vfsd vfs.Dentry
- DentryRefs
+
+ // refs is the reference count. When refs reaches 0, the dentry may be
+ // added to the cache or destroyed. If refs == -1, the dentry has already
+ // been destroyed. refs are allowed to go to 0 and increase again. refs is
+ // accessed using atomic memory operations.
+ refs int64
// fs is the owning filesystem. fs is immutable.
fs *Filesystem
@@ -177,6 +193,12 @@ type Dentry struct {
parent *Dentry
name string
+ // If cached is true, dentryEntry links dentry into
+ // Filesystem.cachedDentries. cached and dentryEntry are protected by
+ // Filesystem.mu.
+ cached bool
+ dentryEntry
+
// dirMu protects children and the names of child Dentries.
//
// Note that holding fs.mu for writing is not sufficient;
@@ -188,6 +210,150 @@ type Dentry struct {
inode Inode
}
+// IncRef implements vfs.DentryImpl.IncRef.
+func (d *Dentry) IncRef() {
+ // d.refs may be 0 if d.fs.mu is locked, which serializes against
+ // d.cacheLocked().
+ atomic.AddInt64(&d.refs, 1)
+}
+
+// TryIncRef implements vfs.DentryImpl.TryIncRef.
+func (d *Dentry) TryIncRef() bool {
+ for {
+ refs := atomic.LoadInt64(&d.refs)
+ if refs <= 0 {
+ return false
+ }
+ if atomic.CompareAndSwapInt64(&d.refs, refs, refs+1) {
+ return true
+ }
+ }
+}
+
+// DecRef implements vfs.DentryImpl.DecRef.
+func (d *Dentry) DecRef(ctx context.Context) {
+ if refs := atomic.AddInt64(&d.refs, -1); refs == 0 {
+ d.fs.mu.Lock()
+ d.cacheLocked(ctx)
+ d.fs.mu.Unlock()
+ } else if refs < 0 {
+ panic("kernfs.Dentry.DecRef() called without holding a reference")
+ }
+}
+
+// cacheLocked should be called after d's reference count becomes 0. The ref
+// count check may happen before acquiring d.fs.mu so there might be a race
+// condition where the ref count is increased again by the time the caller
+// acquires d.fs.mu. This race is handled.
+// Only reachable dentries are added to the cache. However, a dentry might
+// become unreachable *while* it is in the cache due to invalidation.
+//
+// Preconditions: d.fs.mu must be locked for writing.
+func (d *Dentry) cacheLocked(ctx context.Context) {
+ // Dentries with a non-zero reference count must be retained. (The only way
+ // to obtain a reference on a dentry with zero references is via path
+ // resolution, which requires d.fs.mu, so if d.refs is zero then it will
+ // remain zero while we hold d.fs.mu for writing.)
+ refs := atomic.LoadInt64(&d.refs)
+ if refs == -1 {
+ // Dentry has already been destroyed.
+ panic(fmt.Sprintf("cacheLocked called on a dentry which has already been destroyed: %v", d))
+ }
+ if refs > 0 {
+ if d.cached {
+ d.fs.cachedDentries.Remove(d)
+ d.fs.cachedDentriesLen--
+ d.cached = false
+ }
+ return
+ }
+ // If the dentry is deleted and invalidated or has no parent, then it is no
+ // longer reachable by path resolution and should be dropped immediately
+ // because it has zero references.
+ // Note that a dentry may not always have a parent; for example magic links
+ // as described in Inode.Getlink.
+ if isDead := d.VFSDentry().IsDead(); isDead || d.parent == nil {
+ if !isDead {
+ d.fs.vfsfs.VirtualFilesystem().InvalidateDentry(ctx, d.VFSDentry())
+ }
+ if d.cached {
+ d.fs.cachedDentries.Remove(d)
+ d.fs.cachedDentriesLen--
+ d.cached = false
+ }
+ d.destroyLocked(ctx)
+ return
+ }
+ // If d is already cached, just move it to the front of the LRU.
+ if d.cached {
+ d.fs.cachedDentries.Remove(d)
+ d.fs.cachedDentries.PushFront(d)
+ return
+ }
+ // Cache the dentry, then evict the least recently used cached dentry if
+ // the cache becomes over-full.
+ d.fs.cachedDentries.PushFront(d)
+ d.fs.cachedDentriesLen++
+ d.cached = true
+ if d.fs.cachedDentriesLen <= d.fs.MaxCachedDentries {
+ return
+ }
+ // Evict the least recently used dentry because cache size is greater than
+ // max cache size (configured on mount).
+ victim := d.fs.cachedDentries.Back()
+ d.fs.cachedDentries.Remove(victim)
+ d.fs.cachedDentriesLen--
+ victim.cached = false
+ // victim.refs may have become non-zero from an earlier path resolution
+ // after it was inserted into fs.cachedDentries.
+ if atomic.LoadInt64(&victim.refs) == 0 {
+ if !victim.vfsd.IsDead() {
+ victim.parent.dirMu.Lock()
+ // Note that victim can't be a mount point (in any mount
+ // namespace), since VFS holds references on mount points.
+ d.fs.vfsfs.VirtualFilesystem().InvalidateDentry(ctx, victim.VFSDentry())
+ delete(victim.parent.children, victim.name)
+ victim.parent.dirMu.Unlock()
+ }
+ victim.destroyLocked(ctx)
+ }
+ // Whether or not victim was destroyed, we brought fs.cachedDentriesLen
+ // back down to fs.MaxCachedDentries, so we don't loop.
+}
+
+// destroyLocked destroys the dentry.
+//
+// Preconditions:
+// * d.fs.mu must be locked for writing.
+// * d.refs == 0.
+// * d should have been removed from d.parent.children, i.e. d is not reachable
+// by path traversal.
+// * d.vfsd.IsDead() is true.
+func (d *Dentry) destroyLocked(ctx context.Context) {
+ switch atomic.LoadInt64(&d.refs) {
+ case 0:
+ // Mark the dentry destroyed.
+ atomic.StoreInt64(&d.refs, -1)
+ case -1:
+ panic("dentry.destroyLocked() called on already destroyed dentry")
+ default:
+ panic("dentry.destroyLocked() called with references on the dentry")
+ }
+
+ d.inode.DecRef(ctx) // IncRef from Init.
+ d.inode = nil
+
+ // Drop the reference held by d on its parent without recursively locking
+ // d.fs.mu.
+ if d.parent != nil {
+ if refs := atomic.AddInt64(&d.parent.refs, -1); refs == 0 {
+ d.parent.cacheLocked(ctx)
+ } else if refs < 0 {
+ panic("kernfs.Dentry.DecRef() called without holding a reference")
+ }
+ }
+}
+
// Init initializes this dentry.
//
// Precondition: Caller must hold a reference on inode.
@@ -197,6 +363,7 @@ func (d *Dentry) Init(fs *Filesystem, inode Inode) {
d.vfsd.Init(d)
d.fs = fs
d.inode = inode
+ atomic.StoreInt64(&d.refs, 1)
ftype := inode.Mode().FileType()
if ftype == linux.ModeDirectory {
d.flags |= dflagsIsDir
@@ -204,7 +371,6 @@ func (d *Dentry) Init(fs *Filesystem, inode Inode) {
if ftype == linux.ModeSymlink {
d.flags |= dflagsIsSymlink
}
- d.EnableLeakCheck()
}
// VFSDentry returns the generic vfs dentry for this kernfs dentry.
@@ -222,32 +388,6 @@ func (d *Dentry) isSymlink() bool {
return atomic.LoadUint32(&d.flags)&dflagsIsSymlink != 0
}
-// DecRef implements vfs.DentryImpl.DecRef.
-func (d *Dentry) DecRef(ctx context.Context) {
- decRefParent := false
- d.fs.mu.Lock()
- d.DentryRefs.DecRef(func() {
- d.inode.DecRef(ctx) // IncRef from Init.
- d.inode = nil
- if d.parent != nil {
- // We will DecRef d.parent once all locks are dropped.
- decRefParent = true
- d.parent.dirMu.Lock()
- // Remove d from parent.children. It might already have been
- // removed due to invalidation.
- if _, ok := d.parent.children[d.name]; ok {
- delete(d.parent.children, d.name)
- d.fs.VFSFilesystem().VirtualFilesystem().InvalidateDentry(ctx, d.VFSDentry())
- }
- d.parent.dirMu.Unlock()
- }
- })
- d.fs.mu.Unlock()
- if decRefParent {
- d.parent.DecRef(ctx) // IncRef from Dentry.insertChild.
- }
-}
-
// InotifyWithParent implements vfs.DentryImpl.InotifyWithParent.
//
// Although Linux technically supports inotify on pseudo filesystems (inotify
@@ -267,7 +407,9 @@ func (d *Dentry) OnZeroWatches(context.Context) {}
// this dentry. This does not update the directory inode, so calling this on its
// own isn't sufficient to insert a child into a directory.
//
-// Precondition: d must represent a directory inode.
+// Preconditions:
+// * d must represent a directory inode.
+// * d.fs.mu must be locked for at least reading.
func (d *Dentry) insertChild(name string, child *Dentry) {
d.dirMu.Lock()
d.insertChildLocked(name, child)
@@ -280,6 +422,7 @@ func (d *Dentry) insertChild(name string, child *Dentry) {
// Preconditions:
// * d must represent a directory inode.
// * d.dirMu must be locked.
+// * d.fs.mu must be locked for at least reading.
func (d *Dentry) insertChildLocked(name string, child *Dentry) {
if !d.isDir() {
panic(fmt.Sprintf("insertChildLocked called on non-directory Dentry: %+v.", d))
@@ -436,7 +579,7 @@ type inodeDirectory interface {
// the inode is a directory.
//
// The child returned by Lookup will be hashed into the VFS dentry tree,
- // atleast for the duration of the current FS operation.
+ // at least for the duration of the current FS operation.
//
// Lookup must return the child with an extra reference whose ownership is
// transferred to the dentry that is created to point to that inode. If
@@ -454,7 +597,7 @@ type inodeDirectory interface {
// inside the entries returned by this IterDirents invocation. In other words,
// 'offset' should be used to calculate each vfs.Dirent.NextOff as well as
// the return value, while 'relOffset' is the place to start iteration.
- IterDirents(ctx context.Context, callback vfs.IterDirentsCallback, offset, relOffset int64) (newOffset int64, err error)
+ IterDirents(ctx context.Context, mnt *vfs.Mount, callback vfs.IterDirentsCallback, offset, relOffset int64) (newOffset int64, err error)
}
type inodeSymlink interface {
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs_test.go b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
index 82fa19c03..2418eec44 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs_test.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
@@ -36,7 +36,7 @@ const staticFileContent = "This is sample content for a static test file."
// RootDentryFn is a generator function for creating the root dentry of a test
// filesystem. See newTestSystem.
-type RootDentryFn func(*auth.Credentials, *filesystem) kernfs.Inode
+type RootDentryFn func(context.Context, *auth.Credentials, *filesystem) kernfs.Inode
// newTestSystem sets up a minimal environment for running a test, including an
// instance of a test filesystem. Tests can control the contents of the
@@ -72,10 +72,10 @@ type file struct {
content string
}
-func (fs *filesystem) newFile(creds *auth.Credentials, content string) kernfs.Inode {
+func (fs *filesystem) newFile(ctx context.Context, creds *auth.Credentials, content string) kernfs.Inode {
f := &file{}
f.content = content
- f.DynamicBytesFile.Init(creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), f, 0777)
+ f.DynamicBytesFile.Init(ctx, creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), f, 0777)
return f
}
@@ -105,9 +105,9 @@ type readonlyDir struct {
locks vfs.FileLocks
}
-func (fs *filesystem) newReadonlyDir(creds *auth.Credentials, mode linux.FileMode, contents map[string]kernfs.Inode) kernfs.Inode {
+func (fs *filesystem) newReadonlyDir(ctx context.Context, creds *auth.Credentials, mode linux.FileMode, contents map[string]kernfs.Inode) kernfs.Inode {
dir := &readonlyDir{}
- dir.attrs.Init(creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), linux.ModeDirectory|mode)
+ dir.attrs.Init(ctx, creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), linux.ModeDirectory|mode)
dir.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
dir.EnableLeakCheck()
dir.IncLinks(dir.OrderedChildren.Populate(contents))
@@ -142,10 +142,10 @@ type dir struct {
fs *filesystem
}
-func (fs *filesystem) newDir(creds *auth.Credentials, mode linux.FileMode, contents map[string]kernfs.Inode) kernfs.Inode {
+func (fs *filesystem) newDir(ctx context.Context, creds *auth.Credentials, mode linux.FileMode, contents map[string]kernfs.Inode) kernfs.Inode {
dir := &dir{}
dir.fs = fs
- dir.attrs.Init(creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), linux.ModeDirectory|mode)
+ dir.attrs.Init(ctx, creds, 0 /* devMajor */, 0 /* devMinor */, fs.NextIno(), linux.ModeDirectory|mode)
dir.OrderedChildren.Init(kernfs.OrderedChildrenOptions{Writable: true})
dir.EnableLeakCheck()
@@ -169,22 +169,24 @@ func (d *dir) DecRef(ctx context.Context) {
func (d *dir) NewDir(ctx context.Context, name string, opts vfs.MkdirOptions) (kernfs.Inode, error) {
creds := auth.CredentialsFromContext(ctx)
- dir := d.fs.newDir(creds, opts.Mode, nil)
+ dir := d.fs.newDir(ctx, creds, opts.Mode, nil)
if err := d.OrderedChildren.Insert(name, dir); err != nil {
dir.DecRef(ctx)
return nil, err
}
+ d.TouchCMtime(ctx)
d.IncLinks(1)
return dir, nil
}
func (d *dir) NewFile(ctx context.Context, name string, opts vfs.OpenOptions) (kernfs.Inode, error) {
creds := auth.CredentialsFromContext(ctx)
- f := d.fs.newFile(creds, "")
+ f := d.fs.newFile(ctx, creds, "")
if err := d.OrderedChildren.Insert(name, f); err != nil {
f.DecRef(ctx)
return nil, err
}
+ d.TouchCMtime(ctx)
return f, nil
}
@@ -209,7 +211,7 @@ func (fsType) Release(ctx context.Context) {}
func (fst fsType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opt vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
fs := &filesystem{}
fs.VFSFilesystem().Init(vfsObj, &fst, fs)
- root := fst.rootFn(creds, fs)
+ root := fst.rootFn(ctx, creds, fs)
var d kernfs.Dentry
d.Init(&fs.Filesystem, root)
return fs.VFSFilesystem(), d.VFSDentry(), nil
@@ -218,9 +220,9 @@ func (fst fsType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesyst
// -------------------- Remainder of the file are test cases --------------------
func TestBasic(t *testing.T) {
- sys := newTestSystem(t, func(creds *auth.Credentials, fs *filesystem) kernfs.Inode {
- return fs.newReadonlyDir(creds, 0755, map[string]kernfs.Inode{
- "file1": fs.newFile(creds, staticFileContent),
+ sys := newTestSystem(t, func(ctx context.Context, creds *auth.Credentials, fs *filesystem) kernfs.Inode {
+ return fs.newReadonlyDir(ctx, creds, 0755, map[string]kernfs.Inode{
+ "file1": fs.newFile(ctx, creds, staticFileContent),
})
})
defer sys.Destroy()
@@ -228,9 +230,9 @@ func TestBasic(t *testing.T) {
}
func TestMkdirGetDentry(t *testing.T) {
- sys := newTestSystem(t, func(creds *auth.Credentials, fs *filesystem) kernfs.Inode {
- return fs.newReadonlyDir(creds, 0755, map[string]kernfs.Inode{
- "dir1": fs.newDir(creds, 0755, nil),
+ sys := newTestSystem(t, func(ctx context.Context, creds *auth.Credentials, fs *filesystem) kernfs.Inode {
+ return fs.newReadonlyDir(ctx, creds, 0755, map[string]kernfs.Inode{
+ "dir1": fs.newDir(ctx, creds, 0755, nil),
})
})
defer sys.Destroy()
@@ -243,9 +245,9 @@ func TestMkdirGetDentry(t *testing.T) {
}
func TestReadStaticFile(t *testing.T) {
- sys := newTestSystem(t, func(creds *auth.Credentials, fs *filesystem) kernfs.Inode {
- return fs.newReadonlyDir(creds, 0755, map[string]kernfs.Inode{
- "file1": fs.newFile(creds, staticFileContent),
+ sys := newTestSystem(t, func(ctx context.Context, creds *auth.Credentials, fs *filesystem) kernfs.Inode {
+ return fs.newReadonlyDir(ctx, creds, 0755, map[string]kernfs.Inode{
+ "file1": fs.newFile(ctx, creds, staticFileContent),
})
})
defer sys.Destroy()
@@ -269,9 +271,9 @@ func TestReadStaticFile(t *testing.T) {
}
func TestCreateNewFileInStaticDir(t *testing.T) {
- sys := newTestSystem(t, func(creds *auth.Credentials, fs *filesystem) kernfs.Inode {
- return fs.newReadonlyDir(creds, 0755, map[string]kernfs.Inode{
- "dir1": fs.newDir(creds, 0755, nil),
+ sys := newTestSystem(t, func(ctx context.Context, creds *auth.Credentials, fs *filesystem) kernfs.Inode {
+ return fs.newReadonlyDir(ctx, creds, 0755, map[string]kernfs.Inode{
+ "dir1": fs.newDir(ctx, creds, 0755, nil),
})
})
defer sys.Destroy()
@@ -296,8 +298,8 @@ func TestCreateNewFileInStaticDir(t *testing.T) {
}
func TestDirFDReadWrite(t *testing.T) {
- sys := newTestSystem(t, func(creds *auth.Credentials, fs *filesystem) kernfs.Inode {
- return fs.newReadonlyDir(creds, 0755, nil)
+ sys := newTestSystem(t, func(ctx context.Context, creds *auth.Credentials, fs *filesystem) kernfs.Inode {
+ return fs.newReadonlyDir(ctx, creds, 0755, nil)
})
defer sys.Destroy()
@@ -320,14 +322,14 @@ func TestDirFDReadWrite(t *testing.T) {
}
func TestDirFDIterDirents(t *testing.T) {
- sys := newTestSystem(t, func(creds *auth.Credentials, fs *filesystem) kernfs.Inode {
- return fs.newReadonlyDir(creds, 0755, map[string]kernfs.Inode{
+ sys := newTestSystem(t, func(ctx context.Context, creds *auth.Credentials, fs *filesystem) kernfs.Inode {
+ return fs.newReadonlyDir(ctx, creds, 0755, map[string]kernfs.Inode{
// Fill root with nodes backed by various inode implementations.
- "dir1": fs.newReadonlyDir(creds, 0755, nil),
- "dir2": fs.newDir(creds, 0755, map[string]kernfs.Inode{
- "dir3": fs.newDir(creds, 0755, nil),
+ "dir1": fs.newReadonlyDir(ctx, creds, 0755, nil),
+ "dir2": fs.newDir(ctx, creds, 0755, map[string]kernfs.Inode{
+ "dir3": fs.newDir(ctx, creds, 0755, nil),
}),
- "file1": fs.newFile(creds, staticFileContent),
+ "file1": fs.newFile(ctx, creds, staticFileContent),
})
})
defer sys.Destroy()
diff --git a/pkg/sentry/fsimpl/kernfs/symlink.go b/pkg/sentry/fsimpl/kernfs/symlink.go
index 934cc6c9e..a0736c0d6 100644
--- a/pkg/sentry/fsimpl/kernfs/symlink.go
+++ b/pkg/sentry/fsimpl/kernfs/symlink.go
@@ -38,16 +38,16 @@ type StaticSymlink struct {
var _ Inode = (*StaticSymlink)(nil)
// NewStaticSymlink creates a new symlink file pointing to 'target'.
-func NewStaticSymlink(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, target string) Inode {
+func NewStaticSymlink(ctx context.Context, creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, target string) Inode {
inode := &StaticSymlink{}
- inode.Init(creds, devMajor, devMinor, ino, target)
+ inode.Init(ctx, creds, devMajor, devMinor, ino, target)
return inode
}
// Init initializes the instance.
-func (s *StaticSymlink) Init(creds *auth.Credentials, devMajor uint32, devMinor uint32, ino uint64, target string) {
+func (s *StaticSymlink) Init(ctx context.Context, creds *auth.Credentials, devMajor uint32, devMinor uint32, ino uint64, target string) {
s.target = target
- s.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.ModeSymlink|0777)
+ s.InodeAttrs.Init(ctx, creds, devMajor, devMinor, ino, linux.ModeSymlink|0777)
}
// Readlink implements Inode.Readlink.
diff --git a/pkg/sentry/fsimpl/kernfs/synthetic_directory.go b/pkg/sentry/fsimpl/kernfs/synthetic_directory.go
index d0ed17b18..463d77d79 100644
--- a/pkg/sentry/fsimpl/kernfs/synthetic_directory.go
+++ b/pkg/sentry/fsimpl/kernfs/synthetic_directory.go
@@ -41,17 +41,17 @@ type syntheticDirectory struct {
var _ Inode = (*syntheticDirectory)(nil)
-func newSyntheticDirectory(creds *auth.Credentials, perm linux.FileMode) Inode {
+func newSyntheticDirectory(ctx context.Context, creds *auth.Credentials, perm linux.FileMode) Inode {
inode := &syntheticDirectory{}
- inode.Init(creds, 0 /* devMajor */, 0 /* devMinor */, 0 /* ino */, perm)
+ inode.Init(ctx, creds, 0 /* devMajor */, 0 /* devMinor */, 0 /* ino */, perm)
return inode
}
-func (dir *syntheticDirectory) Init(creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode) {
+func (dir *syntheticDirectory) Init(ctx context.Context, creds *auth.Credentials, devMajor, devMinor uint32, ino uint64, perm linux.FileMode) {
if perm&^linux.PermissionsMask != 0 {
panic(fmt.Sprintf("perm contains non-permission bits: %#o", perm))
}
- dir.InodeAttrs.Init(creds, devMajor, devMinor, ino, linux.S_IFDIR|perm)
+ dir.InodeAttrs.Init(ctx, creds, devMajor, devMinor, ino, linux.S_IFDIR|perm)
dir.OrderedChildren.Init(OrderedChildrenOptions{
Writable: true,
})
@@ -76,11 +76,12 @@ func (dir *syntheticDirectory) NewDir(ctx context.Context, name string, opts vfs
if !opts.ForSyntheticMountpoint {
return nil, syserror.EPERM
}
- subdirI := newSyntheticDirectory(auth.CredentialsFromContext(ctx), opts.Mode&linux.PermissionsMask)
+ subdirI := newSyntheticDirectory(ctx, auth.CredentialsFromContext(ctx), opts.Mode&linux.PermissionsMask)
if err := dir.OrderedChildren.Insert(name, subdirI); err != nil {
subdirI.DecRef(ctx)
return nil, err
}
+ dir.TouchCMtime(ctx)
return subdirI, nil
}