diff options
author | gVisor bot <gvisor-bot@google.com> | 2020-10-26 23:18:06 +0000 |
---|---|---|
committer | gVisor bot <gvisor-bot@google.com> | 2020-10-26 23:18:06 +0000 |
commit | 0f69e96daefd4e526de11fba5d04edf5e627e9b4 (patch) | |
tree | f022e7189484dca910cdff37efa4b21300004696 | |
parent | 8ed7429b4634dcfa6cdf39f49299b6292a72631a (diff) | |
parent | 652f11380e44b7c2083fa19d706f5cddf7fe0718 (diff) |
Merge release-20201019.0-63-g652f11380 (automated)
-rw-r--r-- | pkg/sentry/fsimpl/gofer/gofer.go | 11 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/kernfs/dentry_list.go | 193 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/kernfs/dentry_refs.go | 112 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/kernfs/kernfs.go | 196 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/kernfs/kernfs_state_autogen.go | 84 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/proc/filesystem.go | 22 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/sys/sys.go | 22 |
7 files changed, 470 insertions, 170 deletions
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go index 604c6d7cd..80668ebc1 100644 --- a/pkg/sentry/fsimpl/gofer/gofer.go +++ b/pkg/sentry/fsimpl/gofer/gofer.go @@ -153,8 +153,7 @@ type filesystemOptions struct { msize uint32 version string - // maxCachedDentries is the maximum number of dentries with 0 references - // retained by the client. + // maxCachedDentries is the maximum size of filesystem.cachedDentries. maxCachedDentries uint64 // If forcePageCache is true, host FDs may not be used for application @@ -1312,6 +1311,10 @@ func (d *dentry) checkCachingLocked(ctx context.Context) { // resolution, which requires renameMu, so if d.refs is zero then it will // remain zero while we hold renameMu for writing.) refs := atomic.LoadInt64(&d.refs) + if refs == -1 { + // Dentry has already been destroyed. + return + } if refs > 0 { if d.cached { d.fs.cachedDentries.Remove(d) @@ -1320,10 +1323,6 @@ func (d *dentry) checkCachingLocked(ctx context.Context) { } return } - if refs == -1 { - // Dentry has already been destroyed. - return - } // Deleted and invalidated dentries with zero references are no longer // reachable by path resolution and should be dropped immediately. if d.vfsd.IsDead() { diff --git a/pkg/sentry/fsimpl/kernfs/dentry_list.go b/pkg/sentry/fsimpl/kernfs/dentry_list.go new file mode 100644 index 000000000..06101fa32 --- /dev/null +++ b/pkg/sentry/fsimpl/kernfs/dentry_list.go @@ -0,0 +1,193 @@ +package kernfs + +// ElementMapper provides an identity mapping by default. +// +// This can be replaced to provide a struct that maps elements to linker +// objects, if they are not the same. An ElementMapper is not typically +// required if: Linker is left as is, Element is left as is, or Linker and +// Element are the same type. +type dentryElementMapper struct{} + +// linkerFor maps an Element to a Linker. +// +// This default implementation should be inlined. +// +//go:nosplit +func (dentryElementMapper) linkerFor(elem *Dentry) *Dentry { return elem } + +// List is an intrusive list. Entries can be added to or removed from the list +// in O(1) time and with no additional memory allocations. +// +// The zero value for List is an empty list ready to use. +// +// To iterate over a list (where l is a List): +// for e := l.Front(); e != nil; e = e.Next() { +// // do something with e. +// } +// +// +stateify savable +type dentryList struct { + head *Dentry + tail *Dentry +} + +// Reset resets list l to the empty state. +func (l *dentryList) Reset() { + l.head = nil + l.tail = nil +} + +// Empty returns true iff the list is empty. +func (l *dentryList) Empty() bool { + return l.head == nil +} + +// Front returns the first element of list l or nil. +func (l *dentryList) Front() *Dentry { + return l.head +} + +// Back returns the last element of list l or nil. +func (l *dentryList) Back() *Dentry { + return l.tail +} + +// Len returns the number of elements in the list. +// +// NOTE: This is an O(n) operation. +func (l *dentryList) Len() (count int) { + for e := l.Front(); e != nil; e = (dentryElementMapper{}.linkerFor(e)).Next() { + count++ + } + return count +} + +// PushFront inserts the element e at the front of list l. +func (l *dentryList) PushFront(e *Dentry) { + linker := dentryElementMapper{}.linkerFor(e) + linker.SetNext(l.head) + linker.SetPrev(nil) + if l.head != nil { + dentryElementMapper{}.linkerFor(l.head).SetPrev(e) + } else { + l.tail = e + } + + l.head = e +} + +// PushBack inserts the element e at the back of list l. +func (l *dentryList) PushBack(e *Dentry) { + linker := dentryElementMapper{}.linkerFor(e) + linker.SetNext(nil) + linker.SetPrev(l.tail) + if l.tail != nil { + dentryElementMapper{}.linkerFor(l.tail).SetNext(e) + } else { + l.head = e + } + + l.tail = e +} + +// PushBackList inserts list m at the end of list l, emptying m. +func (l *dentryList) PushBackList(m *dentryList) { + if l.head == nil { + l.head = m.head + l.tail = m.tail + } else if m.head != nil { + dentryElementMapper{}.linkerFor(l.tail).SetNext(m.head) + dentryElementMapper{}.linkerFor(m.head).SetPrev(l.tail) + + l.tail = m.tail + } + m.head = nil + m.tail = nil +} + +// InsertAfter inserts e after b. +func (l *dentryList) InsertAfter(b, e *Dentry) { + bLinker := dentryElementMapper{}.linkerFor(b) + eLinker := dentryElementMapper{}.linkerFor(e) + + a := bLinker.Next() + + eLinker.SetNext(a) + eLinker.SetPrev(b) + bLinker.SetNext(e) + + if a != nil { + dentryElementMapper{}.linkerFor(a).SetPrev(e) + } else { + l.tail = e + } +} + +// InsertBefore inserts e before a. +func (l *dentryList) InsertBefore(a, e *Dentry) { + aLinker := dentryElementMapper{}.linkerFor(a) + eLinker := dentryElementMapper{}.linkerFor(e) + + b := aLinker.Prev() + eLinker.SetNext(a) + eLinker.SetPrev(b) + aLinker.SetPrev(e) + + if b != nil { + dentryElementMapper{}.linkerFor(b).SetNext(e) + } else { + l.head = e + } +} + +// Remove removes e from l. +func (l *dentryList) Remove(e *Dentry) { + linker := dentryElementMapper{}.linkerFor(e) + prev := linker.Prev() + next := linker.Next() + + if prev != nil { + dentryElementMapper{}.linkerFor(prev).SetNext(next) + } else if l.head == e { + l.head = next + } + + if next != nil { + dentryElementMapper{}.linkerFor(next).SetPrev(prev) + } else if l.tail == e { + l.tail = prev + } + + linker.SetNext(nil) + linker.SetPrev(nil) +} + +// Entry is a default implementation of Linker. Users can add anonymous fields +// of this type to their structs to make them automatically implement the +// methods needed by List. +// +// +stateify savable +type dentryEntry struct { + next *Dentry + prev *Dentry +} + +// Next returns the entry that follows e in the list. +func (e *dentryEntry) Next() *Dentry { + return e.next +} + +// Prev returns the entry that precedes e in the list. +func (e *dentryEntry) Prev() *Dentry { + return e.prev +} + +// SetNext assigns 'entry' as the entry that follows e in the list. +func (e *dentryEntry) SetNext(elem *Dentry) { + e.next = elem +} + +// SetPrev assigns 'entry' as the entry that precedes e in the list. +func (e *dentryEntry) SetPrev(elem *Dentry) { + e.prev = elem +} diff --git a/pkg/sentry/fsimpl/kernfs/dentry_refs.go b/pkg/sentry/fsimpl/kernfs/dentry_refs.go deleted file mode 100644 index c2304939b..000000000 --- a/pkg/sentry/fsimpl/kernfs/dentry_refs.go +++ /dev/null @@ -1,112 +0,0 @@ -package kernfs - -import ( - "fmt" - "sync/atomic" - - "gvisor.dev/gvisor/pkg/refsvfs2" -) - -// ownerType is used to customize logging. Note that we use a pointer to T so -// that we do not copy the entire object when passed as a format parameter. -var DentryownerType *Dentry - -// Refs implements refs.RefCounter. It keeps a reference count using atomic -// operations and calls the destructor when the count reaches zero. -// -// Note that the number of references is actually refCount + 1 so that a default -// zero-value Refs object contains one reference. -// -// +stateify savable -type DentryRefs struct { - // refCount is composed of two fields: - // - // [32-bit speculative references]:[32-bit real references] - // - // Speculative references are used for TryIncRef, to avoid a CompareAndSwap - // loop. See IncRef, DecRef and TryIncRef for details of how these fields are - // used. - refCount int64 -} - -// EnableLeakCheck enables reference leak checking on r. -func (r *DentryRefs) EnableLeakCheck() { - if refsvfs2.LeakCheckEnabled() { - refsvfs2.Register(r, fmt.Sprintf("%T", DentryownerType)) - } -} - -// LeakMessage implements refsvfs2.CheckedObject.LeakMessage. -func (r *DentryRefs) LeakMessage() string { - return fmt.Sprintf("%T %p: reference count of %d instead of 0", DentryownerType, r, r.ReadRefs()) -} - -// ReadRefs returns the current number of references. The returned count is -// inherently racy and is unsafe to use without external synchronization. -func (r *DentryRefs) ReadRefs() int64 { - - return atomic.LoadInt64(&r.refCount) + 1 -} - -// IncRef implements refs.RefCounter.IncRef. -// -//go:nosplit -func (r *DentryRefs) IncRef() { - if v := atomic.AddInt64(&r.refCount, 1); v <= 0 { - panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, DentryownerType)) - } -} - -// TryIncRef implements refs.RefCounter.TryIncRef. -// -// To do this safely without a loop, a speculative reference is first acquired -// on the object. This allows multiple concurrent TryIncRef calls to distinguish -// other TryIncRef calls from genuine references held. -// -//go:nosplit -func (r *DentryRefs) TryIncRef() bool { - const speculativeRef = 1 << 32 - v := atomic.AddInt64(&r.refCount, speculativeRef) - if int32(v) < 0 { - - atomic.AddInt64(&r.refCount, -speculativeRef) - return false - } - - atomic.AddInt64(&r.refCount, -speculativeRef+1) - return true -} - -// DecRef implements refs.RefCounter.DecRef. -// -// Note that speculative references are counted here. Since they were added -// prior to real references reaching zero, they will successfully convert to -// real references. In other words, we see speculative references only in the -// following case: -// -// A: TryIncRef [speculative increase => sees non-negative references] -// B: DecRef [real decrease] -// A: TryIncRef [transform speculative to real] -// -//go:nosplit -func (r *DentryRefs) DecRef(destroy func()) { - switch v := atomic.AddInt64(&r.refCount, -1); { - case v < -1: - panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, DentryownerType)) - - case v == -1: - if refsvfs2.LeakCheckEnabled() { - refsvfs2.Unregister(r, fmt.Sprintf("%T", DentryownerType)) - } - - if destroy != nil { - destroy() - } - } -} - -func (r *DentryRefs) afterLoad() { - if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 { - r.EnableLeakCheck() - } -} diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go index 25184dbf9..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 diff --git a/pkg/sentry/fsimpl/kernfs/kernfs_state_autogen.go b/pkg/sentry/fsimpl/kernfs/kernfs_state_autogen.go index 4a00f2b37..1a62f05aa 100644 --- a/pkg/sentry/fsimpl/kernfs/kernfs_state_autogen.go +++ b/pkg/sentry/fsimpl/kernfs/kernfs_state_autogen.go @@ -6,26 +6,56 @@ import ( "gvisor.dev/gvisor/pkg/state" ) -func (r *DentryRefs) StateTypeName() string { - return "pkg/sentry/fsimpl/kernfs.DentryRefs" +func (l *dentryList) StateTypeName() string { + return "pkg/sentry/fsimpl/kernfs.dentryList" } -func (r *DentryRefs) StateFields() []string { +func (l *dentryList) StateFields() []string { return []string{ - "refCount", + "head", + "tail", } } -func (r *DentryRefs) beforeSave() {} +func (l *dentryList) beforeSave() {} -func (r *DentryRefs) StateSave(stateSinkObject state.Sink) { - r.beforeSave() - stateSinkObject.Save(0, &r.refCount) +func (l *dentryList) StateSave(stateSinkObject state.Sink) { + l.beforeSave() + stateSinkObject.Save(0, &l.head) + stateSinkObject.Save(1, &l.tail) } -func (r *DentryRefs) StateLoad(stateSourceObject state.Source) { - stateSourceObject.Load(0, &r.refCount) - stateSourceObject.AfterLoad(r.afterLoad) +func (l *dentryList) afterLoad() {} + +func (l *dentryList) StateLoad(stateSourceObject state.Source) { + stateSourceObject.Load(0, &l.head) + stateSourceObject.Load(1, &l.tail) +} + +func (e *dentryEntry) StateTypeName() string { + return "pkg/sentry/fsimpl/kernfs.dentryEntry" +} + +func (e *dentryEntry) StateFields() []string { + return []string{ + "next", + "prev", + } +} + +func (e *dentryEntry) beforeSave() {} + +func (e *dentryEntry) StateSave(stateSinkObject state.Sink) { + e.beforeSave() + stateSinkObject.Save(0, &e.next) + stateSinkObject.Save(1, &e.prev) +} + +func (e *dentryEntry) afterLoad() {} + +func (e *dentryEntry) StateLoad(stateSourceObject state.Source) { + stateSourceObject.Load(0, &e.next) + stateSourceObject.Load(1, &e.prev) } func (f *DynamicBytesFile) StateTypeName() string { @@ -555,6 +585,9 @@ func (fs *Filesystem) StateFields() []string { "vfsfs", "droppedDentries", "nextInoMinusOne", + "cachedDentries", + "cachedDentriesLen", + "MaxCachedDentries", } } @@ -565,6 +598,9 @@ func (fs *Filesystem) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(0, &fs.vfsfs) stateSinkObject.Save(1, &fs.droppedDentries) stateSinkObject.Save(2, &fs.nextInoMinusOne) + stateSinkObject.Save(3, &fs.cachedDentries) + stateSinkObject.Save(4, &fs.cachedDentriesLen) + stateSinkObject.Save(5, &fs.MaxCachedDentries) } func (fs *Filesystem) afterLoad() {} @@ -573,6 +609,9 @@ func (fs *Filesystem) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &fs.vfsfs) stateSourceObject.Load(1, &fs.droppedDentries) stateSourceObject.Load(2, &fs.nextInoMinusOne) + stateSourceObject.Load(3, &fs.cachedDentries) + stateSourceObject.Load(4, &fs.cachedDentriesLen) + stateSourceObject.Load(5, &fs.MaxCachedDentries) } func (d *Dentry) StateTypeName() string { @@ -582,11 +621,13 @@ func (d *Dentry) StateTypeName() string { func (d *Dentry) StateFields() []string { return []string{ "vfsd", - "DentryRefs", + "refs", "fs", "flags", "parent", "name", + "cached", + "dentryEntry", "children", "inode", } @@ -597,26 +638,30 @@ func (d *Dentry) beforeSave() {} func (d *Dentry) StateSave(stateSinkObject state.Sink) { d.beforeSave() stateSinkObject.Save(0, &d.vfsd) - stateSinkObject.Save(1, &d.DentryRefs) + stateSinkObject.Save(1, &d.refs) stateSinkObject.Save(2, &d.fs) stateSinkObject.Save(3, &d.flags) stateSinkObject.Save(4, &d.parent) stateSinkObject.Save(5, &d.name) - stateSinkObject.Save(6, &d.children) - stateSinkObject.Save(7, &d.inode) + stateSinkObject.Save(6, &d.cached) + stateSinkObject.Save(7, &d.dentryEntry) + stateSinkObject.Save(8, &d.children) + stateSinkObject.Save(9, &d.inode) } func (d *Dentry) afterLoad() {} func (d *Dentry) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &d.vfsd) - stateSourceObject.Load(1, &d.DentryRefs) + stateSourceObject.Load(1, &d.refs) stateSourceObject.Load(2, &d.fs) stateSourceObject.Load(3, &d.flags) stateSourceObject.Load(4, &d.parent) stateSourceObject.Load(5, &d.name) - stateSourceObject.Load(6, &d.children) - stateSourceObject.Load(7, &d.inode) + stateSourceObject.Load(6, &d.cached) + stateSourceObject.Load(7, &d.dentryEntry) + stateSourceObject.Load(8, &d.children) + stateSourceObject.Load(9, &d.inode) } func (l *slotList) StateTypeName() string { @@ -792,7 +837,8 @@ func (r *syntheticDirectoryRefs) StateLoad(stateSourceObject state.Source) { } func init() { - state.Register((*DentryRefs)(nil)) + state.Register((*dentryList)(nil)) + state.Register((*dentryEntry)(nil)) state.Register((*DynamicBytesFile)(nil)) state.Register((*DynamicBytesFD)(nil)) state.Register((*SeekEndConfig)(nil)) diff --git a/pkg/sentry/fsimpl/proc/filesystem.go b/pkg/sentry/fsimpl/proc/filesystem.go index ba809b92d..99abcab66 100644 --- a/pkg/sentry/fsimpl/proc/filesystem.go +++ b/pkg/sentry/fsimpl/proc/filesystem.go @@ -17,6 +17,7 @@ package proc import ( "fmt" + "strconv" "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" @@ -24,10 +25,14 @@ import ( "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" ) -// Name is the default filesystem name. -const Name = "proc" +const ( + // Name is the default filesystem name. + Name = "proc" + defaultMaxCachedDentries = uint64(1000) +) // FilesystemType is the factory class for procfs. // @@ -63,9 +68,22 @@ func (ft FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualF if err != nil { return nil, nil, err } + + mopts := vfs.GenericParseMountOptions(opts.Data) + maxCachedDentries := defaultMaxCachedDentries + if str, ok := mopts["dentry_cache_limit"]; ok { + delete(mopts, "dentry_cache_limit") + maxCachedDentries, err = strconv.ParseUint(str, 10, 64) + if err != nil { + ctx.Warningf("proc.FilesystemType.GetFilesystem: invalid dentry cache limit: dentry_cache_limit=%s", str) + return nil, nil, syserror.EINVAL + } + } + procfs := &filesystem{ devMinor: devMinor, } + procfs.MaxCachedDentries = maxCachedDentries procfs.VFSFilesystem().Init(vfsObj, &ft, procfs) var cgroups map[string]string diff --git a/pkg/sentry/fsimpl/sys/sys.go b/pkg/sentry/fsimpl/sys/sys.go index 54fa5eced..7d2147141 100644 --- a/pkg/sentry/fsimpl/sys/sys.go +++ b/pkg/sentry/fsimpl/sys/sys.go @@ -18,6 +18,7 @@ package sys import ( "bytes" "fmt" + "strconv" "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" @@ -29,9 +30,12 @@ import ( "gvisor.dev/gvisor/pkg/syserror" ) -// Name is the default filesystem name. -const Name = "sysfs" -const defaultSysDirMode = linux.FileMode(0755) +const ( + // Name is the default filesystem name. + Name = "sysfs" + defaultSysDirMode = linux.FileMode(0755) + defaultMaxCachedDentries = uint64(1000) +) // FilesystemType implements vfs.FilesystemType. // @@ -62,9 +66,21 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt return nil, nil, err } + mopts := vfs.GenericParseMountOptions(opts.Data) + maxCachedDentries := defaultMaxCachedDentries + if str, ok := mopts["dentry_cache_limit"]; ok { + delete(mopts, "dentry_cache_limit") + maxCachedDentries, err = strconv.ParseUint(str, 10, 64) + if err != nil { + ctx.Warningf("sys.FilesystemType.GetFilesystem: invalid dentry cache limit: dentry_cache_limit=%s", str) + return nil, nil, syserror.EINVAL + } + } + fs := &filesystem{ devMinor: devMinor, } + fs.MaxCachedDentries = maxCachedDentries fs.VFSFilesystem().Init(vfsObj, &fsType, fs) root := fs.newDir(ctx, creds, defaultSysDirMode, map[string]kernfs.Inode{ |