summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/vfs
diff options
context:
space:
mode:
authorNayana Bidari <nybidari@google.com>2020-08-03 13:33:47 -0700
committergVisor bot <gvisor-bot@google.com>2020-08-03 13:36:05 -0700
commitb2ae7ea1bb207eddadd7962080e7bd0b8634db96 (patch)
tree7230b0b327debfcdfdd96b132cea0a90181f8281 /pkg/sentry/vfs
parentef11bb936b2bbb50b0ceeeb93a74b94680fff724 (diff)
Plumbing context.Context to DecRef() and Release().
context is passed to DecRef() and Release() which is needed for SO_LINGER implementation. PiperOrigin-RevId: 324672584
Diffstat (limited to 'pkg/sentry/vfs')
-rw-r--r--pkg/sentry/vfs/anonfs.go8
-rw-r--r--pkg/sentry/vfs/dentry.go37
-rw-r--r--pkg/sentry/vfs/epoll.go6
-rw-r--r--pkg/sentry/vfs/file_description.go24
-rw-r--r--pkg/sentry/vfs/file_description_impl_util_test.go18
-rw-r--r--pkg/sentry/vfs/filesystem.go6
-rw-r--r--pkg/sentry/vfs/inotify.go34
-rw-r--r--pkg/sentry/vfs/mount.go54
-rw-r--r--pkg/sentry/vfs/pathname.go18
-rw-r--r--pkg/sentry/vfs/resolving_path.go43
-rw-r--r--pkg/sentry/vfs/vfs.go160
11 files changed, 205 insertions, 203 deletions
diff --git a/pkg/sentry/vfs/anonfs.go b/pkg/sentry/vfs/anonfs.go
index 641e3e502..5a0e3e6b5 100644
--- a/pkg/sentry/vfs/anonfs.go
+++ b/pkg/sentry/vfs/anonfs.go
@@ -82,7 +82,7 @@ type anonDentry struct {
}
// Release implements FilesystemImpl.Release.
-func (fs *anonFilesystem) Release() {
+func (fs *anonFilesystem) Release(ctx context.Context) {
}
// Sync implements FilesystemImpl.Sync.
@@ -294,7 +294,7 @@ func (d *anonDentry) TryIncRef() bool {
}
// DecRef implements DentryImpl.DecRef.
-func (d *anonDentry) DecRef() {
+func (d *anonDentry) DecRef(ctx context.Context) {
// no-op
}
@@ -303,7 +303,7 @@ func (d *anonDentry) DecRef() {
// Although Linux technically supports inotify on pseudo filesystems (inotify
// is implemented at the vfs layer), it is not particularly useful. It is left
// unimplemented until someone actually needs it.
-func (d *anonDentry) InotifyWithParent(events, cookie uint32, et EventType) {}
+func (d *anonDentry) InotifyWithParent(ctx context.Context, events, cookie uint32, et EventType) {}
// Watches implements DentryImpl.Watches.
func (d *anonDentry) Watches() *Watches {
@@ -311,4 +311,4 @@ func (d *anonDentry) Watches() *Watches {
}
// OnZeroWatches implements Dentry.OnZeroWatches.
-func (d *anonDentry) OnZeroWatches() {}
+func (d *anonDentry) OnZeroWatches(context.Context) {}
diff --git a/pkg/sentry/vfs/dentry.go b/pkg/sentry/vfs/dentry.go
index cea3e6955..bc7ea93ea 100644
--- a/pkg/sentry/vfs/dentry.go
+++ b/pkg/sentry/vfs/dentry.go
@@ -17,6 +17,7 @@ package vfs
import (
"sync/atomic"
+ "gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -102,7 +103,7 @@ type DentryImpl interface {
TryIncRef() bool
// DecRef decrements the Dentry's reference count.
- DecRef()
+ DecRef(ctx context.Context)
// InotifyWithParent notifies all watches on the targets represented by this
// dentry and its parent. The parent's watches are notified first, followed
@@ -113,7 +114,7 @@ type DentryImpl interface {
//
// Note that the events may not actually propagate up to the user, depending
// on the event masks.
- InotifyWithParent(events, cookie uint32, et EventType)
+ InotifyWithParent(ctx context.Context, events, cookie uint32, et EventType)
// Watches returns the set of inotify watches for the file corresponding to
// the Dentry. Dentries that are hard links to the same underlying file
@@ -135,7 +136,7 @@ type DentryImpl interface {
// The caller does not need to hold a reference on the dentry. OnZeroWatches
// may acquire inotify locks, so to prevent deadlock, no inotify locks should
// be held by the caller.
- OnZeroWatches()
+ OnZeroWatches(ctx context.Context)
}
// IncRef increments d's reference count.
@@ -150,8 +151,8 @@ func (d *Dentry) TryIncRef() bool {
}
// DecRef decrements d's reference count.
-func (d *Dentry) DecRef() {
- d.impl.DecRef()
+func (d *Dentry) DecRef(ctx context.Context) {
+ d.impl.DecRef(ctx)
}
// IsDead returns true if d has been deleted or invalidated by its owning
@@ -168,8 +169,8 @@ func (d *Dentry) isMounted() bool {
// InotifyWithParent notifies all watches on the targets represented by d and
// its parent of events.
-func (d *Dentry) InotifyWithParent(events, cookie uint32, et EventType) {
- d.impl.InotifyWithParent(events, cookie, et)
+func (d *Dentry) InotifyWithParent(ctx context.Context, events, cookie uint32, et EventType) {
+ d.impl.InotifyWithParent(ctx, events, cookie, et)
}
// Watches returns the set of inotify watches associated with d.
@@ -182,8 +183,8 @@ func (d *Dentry) Watches() *Watches {
// OnZeroWatches performs cleanup tasks whenever the number of watches on a
// dentry drops to zero.
-func (d *Dentry) OnZeroWatches() {
- d.impl.OnZeroWatches()
+func (d *Dentry) OnZeroWatches(ctx context.Context) {
+ d.impl.OnZeroWatches(ctx)
}
// The following functions are exported so that filesystem implementations can
@@ -214,11 +215,11 @@ func (vfs *VirtualFilesystem) AbortDeleteDentry(d *Dentry) {
// CommitDeleteDentry must be called after PrepareDeleteDentry if the deletion
// succeeds.
-func (vfs *VirtualFilesystem) CommitDeleteDentry(d *Dentry) {
+func (vfs *VirtualFilesystem) CommitDeleteDentry(ctx context.Context, d *Dentry) {
d.dead = true
d.mu.Unlock()
if d.isMounted() {
- vfs.forgetDeadMountpoint(d)
+ vfs.forgetDeadMountpoint(ctx, d)
}
}
@@ -226,12 +227,12 @@ func (vfs *VirtualFilesystem) CommitDeleteDentry(d *Dentry) {
// did for reasons outside of VFS' control (e.g. d represents the local state
// of a file on a remote filesystem on which the file has already been
// deleted).
-func (vfs *VirtualFilesystem) InvalidateDentry(d *Dentry) {
+func (vfs *VirtualFilesystem) InvalidateDentry(ctx context.Context, d *Dentry) {
d.mu.Lock()
d.dead = true
d.mu.Unlock()
if d.isMounted() {
- vfs.forgetDeadMountpoint(d)
+ vfs.forgetDeadMountpoint(ctx, d)
}
}
@@ -278,13 +279,13 @@ func (vfs *VirtualFilesystem) AbortRenameDentry(from, to *Dentry) {
// that was replaced by from.
//
// Preconditions: PrepareRenameDentry was previously called on from and to.
-func (vfs *VirtualFilesystem) CommitRenameReplaceDentry(from, to *Dentry) {
+func (vfs *VirtualFilesystem) CommitRenameReplaceDentry(ctx context.Context, from, to *Dentry) {
from.mu.Unlock()
if to != nil {
to.dead = true
to.mu.Unlock()
if to.isMounted() {
- vfs.forgetDeadMountpoint(to)
+ vfs.forgetDeadMountpoint(ctx, to)
}
}
}
@@ -303,7 +304,7 @@ func (vfs *VirtualFilesystem) CommitRenameExchangeDentry(from, to *Dentry) {
//
// forgetDeadMountpoint is analogous to Linux's
// fs/namespace.c:__detach_mounts().
-func (vfs *VirtualFilesystem) forgetDeadMountpoint(d *Dentry) {
+func (vfs *VirtualFilesystem) forgetDeadMountpoint(ctx context.Context, d *Dentry) {
var (
vdsToDecRef []VirtualDentry
mountsToDecRef []*Mount
@@ -316,9 +317,9 @@ func (vfs *VirtualFilesystem) forgetDeadMountpoint(d *Dentry) {
vfs.mounts.seq.EndWrite()
vfs.mountMu.Unlock()
for _, vd := range vdsToDecRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
for _, mnt := range mountsToDecRef {
- mnt.DecRef()
+ mnt.DecRef(ctx)
}
}
diff --git a/pkg/sentry/vfs/epoll.go b/pkg/sentry/vfs/epoll.go
index 5b009b928..1b5af9f73 100644
--- a/pkg/sentry/vfs/epoll.go
+++ b/pkg/sentry/vfs/epoll.go
@@ -93,9 +93,9 @@ type epollInterest struct {
// NewEpollInstanceFD returns a FileDescription representing a new epoll
// instance. A reference is taken on the returned FileDescription.
-func (vfs *VirtualFilesystem) NewEpollInstanceFD() (*FileDescription, error) {
+func (vfs *VirtualFilesystem) NewEpollInstanceFD(ctx context.Context) (*FileDescription, error) {
vd := vfs.NewAnonVirtualDentry("[eventpoll]")
- defer vd.DecRef()
+ defer vd.DecRef(ctx)
ep := &EpollInstance{
interest: make(map[epollInterestKey]*epollInterest),
}
@@ -110,7 +110,7 @@ func (vfs *VirtualFilesystem) NewEpollInstanceFD() (*FileDescription, error) {
}
// Release implements FileDescriptionImpl.Release.
-func (ep *EpollInstance) Release() {
+func (ep *EpollInstance) Release(ctx context.Context) {
// Unregister all polled fds.
ep.interestMu.Lock()
defer ep.interestMu.Unlock()
diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go
index 93861fb4a..576ab3920 100644
--- a/pkg/sentry/vfs/file_description.go
+++ b/pkg/sentry/vfs/file_description.go
@@ -171,7 +171,7 @@ func (fd *FileDescription) TryIncRef() bool {
}
// DecRef decrements fd's reference count.
-func (fd *FileDescription) DecRef() {
+func (fd *FileDescription) DecRef(ctx context.Context) {
if refs := atomic.AddInt64(&fd.refs, -1); refs == 0 {
// Unregister fd from all epoll instances.
fd.epollMu.Lock()
@@ -196,11 +196,11 @@ func (fd *FileDescription) DecRef() {
}
// Release implementation resources.
- fd.impl.Release()
+ fd.impl.Release(ctx)
if fd.writable {
fd.vd.mount.EndWrite()
}
- fd.vd.DecRef()
+ fd.vd.DecRef(ctx)
fd.flagsMu.Lock()
// TODO(gvisor.dev/issue/1663): We may need to unregister during save, as we do in VFS1.
if fd.statusFlags&linux.O_ASYNC != 0 && fd.asyncHandler != nil {
@@ -335,7 +335,7 @@ func (fd *FileDescription) Impl() FileDescriptionImpl {
type FileDescriptionImpl interface {
// Release is called when the associated FileDescription reaches zero
// references.
- Release()
+ Release(ctx context.Context)
// OnClose is called when a file descriptor representing the
// FileDescription is closed. Note that returning a non-nil error does not
@@ -526,7 +526,7 @@ func (fd *FileDescription) Stat(ctx context.Context, opts StatOptions) (linux.St
Start: fd.vd,
})
stat, err := fd.vd.mount.fs.impl.StatAt(ctx, rp, opts)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return stat, err
}
return fd.impl.Stat(ctx, opts)
@@ -541,7 +541,7 @@ func (fd *FileDescription) SetStat(ctx context.Context, opts SetStatOptions) err
Start: fd.vd,
})
err := fd.vd.mount.fs.impl.SetStatAt(ctx, rp, opts)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return err
}
return fd.impl.SetStat(ctx, opts)
@@ -557,7 +557,7 @@ func (fd *FileDescription) StatFS(ctx context.Context) (linux.Statfs, error) {
Start: fd.vd,
})
statfs, err := fd.vd.mount.fs.impl.StatFSAt(ctx, rp)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return statfs, err
}
return fd.impl.StatFS(ctx)
@@ -674,7 +674,7 @@ func (fd *FileDescription) Listxattr(ctx context.Context, size uint64) ([]string
Start: fd.vd,
})
names, err := fd.vd.mount.fs.impl.ListxattrAt(ctx, rp, size)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return names, err
}
names, err := fd.impl.Listxattr(ctx, size)
@@ -703,7 +703,7 @@ func (fd *FileDescription) Getxattr(ctx context.Context, opts *GetxattrOptions)
Start: fd.vd,
})
val, err := fd.vd.mount.fs.impl.GetxattrAt(ctx, rp, *opts)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return val, err
}
return fd.impl.Getxattr(ctx, *opts)
@@ -719,7 +719,7 @@ func (fd *FileDescription) Setxattr(ctx context.Context, opts *SetxattrOptions)
Start: fd.vd,
})
err := fd.vd.mount.fs.impl.SetxattrAt(ctx, rp, *opts)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return err
}
return fd.impl.Setxattr(ctx, *opts)
@@ -735,7 +735,7 @@ func (fd *FileDescription) Removexattr(ctx context.Context, name string) error {
Start: fd.vd,
})
err := fd.vd.mount.fs.impl.RemovexattrAt(ctx, rp, name)
- vfsObj.putResolvingPath(rp)
+ vfsObj.putResolvingPath(ctx, rp)
return err
}
return fd.impl.Removexattr(ctx, name)
@@ -752,7 +752,7 @@ func (fd *FileDescription) MappedName(ctx context.Context) string {
vfsroot := RootFromContext(ctx)
s, _ := fd.vd.mount.vfs.PathnameWithDeleted(ctx, vfsroot, fd.vd)
if vfsroot.Ok() {
- vfsroot.DecRef()
+ vfsroot.DecRef(ctx)
}
return s
}
diff --git a/pkg/sentry/vfs/file_description_impl_util_test.go b/pkg/sentry/vfs/file_description_impl_util_test.go
index 3b7e1c273..1cd607c0a 100644
--- a/pkg/sentry/vfs/file_description_impl_util_test.go
+++ b/pkg/sentry/vfs/file_description_impl_util_test.go
@@ -80,9 +80,9 @@ type testFD struct {
data DynamicBytesSource
}
-func newTestFD(vfsObj *VirtualFilesystem, statusFlags uint32, data DynamicBytesSource) *FileDescription {
+func newTestFD(ctx context.Context, vfsObj *VirtualFilesystem, statusFlags uint32, data DynamicBytesSource) *FileDescription {
vd := vfsObj.NewAnonVirtualDentry("genCountFD")
- defer vd.DecRef()
+ defer vd.DecRef(ctx)
var fd testFD
fd.vfsfd.Init(&fd, statusFlags, vd.Mount(), vd.Dentry(), &FileDescriptionOptions{})
fd.DynamicBytesFileDescriptionImpl.SetDataSource(data)
@@ -90,7 +90,7 @@ func newTestFD(vfsObj *VirtualFilesystem, statusFlags uint32, data DynamicBytesS
}
// Release implements FileDescriptionImpl.Release.
-func (fd *testFD) Release() {
+func (fd *testFD) Release(context.Context) {
}
// SetStatusFlags implements FileDescriptionImpl.SetStatusFlags.
@@ -109,11 +109,11 @@ func TestGenCountFD(t *testing.T) {
ctx := contexttest.Context(t)
vfsObj := &VirtualFilesystem{}
- if err := vfsObj.Init(); err != nil {
+ if err := vfsObj.Init(ctx); err != nil {
t.Fatalf("VFS init: %v", err)
}
- fd := newTestFD(vfsObj, linux.O_RDWR, &genCount{})
- defer fd.DecRef()
+ fd := newTestFD(ctx, vfsObj, linux.O_RDWR, &genCount{})
+ defer fd.DecRef(ctx)
// The first read causes Generate to be called to fill the FD's buffer.
buf := make([]byte, 2)
@@ -167,11 +167,11 @@ func TestWritable(t *testing.T) {
ctx := contexttest.Context(t)
vfsObj := &VirtualFilesystem{}
- if err := vfsObj.Init(); err != nil {
+ if err := vfsObj.Init(ctx); err != nil {
t.Fatalf("VFS init: %v", err)
}
- fd := newTestFD(vfsObj, linux.O_RDWR, &storeData{data: "init"})
- defer fd.DecRef()
+ fd := newTestFD(ctx, vfsObj, linux.O_RDWR, &storeData{data: "init"})
+ defer fd.DecRef(ctx)
buf := make([]byte, 10)
ioseq := usermem.BytesIOSequence(buf)
diff --git a/pkg/sentry/vfs/filesystem.go b/pkg/sentry/vfs/filesystem.go
index 6bb9ca180..df3758fd1 100644
--- a/pkg/sentry/vfs/filesystem.go
+++ b/pkg/sentry/vfs/filesystem.go
@@ -100,12 +100,12 @@ func (fs *Filesystem) TryIncRef() bool {
}
// DecRef decrements fs' reference count.
-func (fs *Filesystem) DecRef() {
+func (fs *Filesystem) DecRef(ctx context.Context) {
if refs := atomic.AddInt64(&fs.refs, -1); refs == 0 {
fs.vfs.filesystemsMu.Lock()
delete(fs.vfs.filesystems, fs)
fs.vfs.filesystemsMu.Unlock()
- fs.impl.Release()
+ fs.impl.Release(ctx)
} else if refs < 0 {
panic("Filesystem.decRef() called without holding a reference")
}
@@ -149,7 +149,7 @@ func (fs *Filesystem) DecRef() {
type FilesystemImpl interface {
// Release is called when the associated Filesystem reaches zero
// references.
- Release()
+ Release(ctx context.Context)
// Sync "causes all pending modifications to filesystem metadata and cached
// file data to be written to the underlying [filesystem]", as by syncfs(2).
diff --git a/pkg/sentry/vfs/inotify.go b/pkg/sentry/vfs/inotify.go
index 167b731ac..aff220a61 100644
--- a/pkg/sentry/vfs/inotify.go
+++ b/pkg/sentry/vfs/inotify.go
@@ -100,7 +100,7 @@ func NewInotifyFD(ctx context.Context, vfsObj *VirtualFilesystem, flags uint32)
id := uniqueid.GlobalFromContext(ctx)
vd := vfsObj.NewAnonVirtualDentry(fmt.Sprintf("[inotifyfd:%d]", id))
- defer vd.DecRef()
+ defer vd.DecRef(ctx)
fd := &Inotify{
id: id,
scratch: make([]byte, inotifyEventBaseSize),
@@ -118,7 +118,7 @@ func NewInotifyFD(ctx context.Context, vfsObj *VirtualFilesystem, flags uint32)
// Release implements FileDescriptionImpl.Release. Release removes all
// watches and frees all resources for an inotify instance.
-func (i *Inotify) Release() {
+func (i *Inotify) Release(ctx context.Context) {
var ds []*Dentry
// We need to hold i.mu to avoid a race with concurrent calls to
@@ -144,7 +144,7 @@ func (i *Inotify) Release() {
i.mu.Unlock()
for _, d := range ds {
- d.OnZeroWatches()
+ d.OnZeroWatches(ctx)
}
}
@@ -350,7 +350,7 @@ func (i *Inotify) AddWatch(target *Dentry, mask uint32) (int32, error) {
// RmWatch looks up an inotify watch for the given 'wd' and configures the
// target to stop sending events to this inotify instance.
-func (i *Inotify) RmWatch(wd int32) error {
+func (i *Inotify) RmWatch(ctx context.Context, wd int32) error {
i.mu.Lock()
// Find the watch we were asked to removed.
@@ -374,7 +374,7 @@ func (i *Inotify) RmWatch(wd int32) error {
i.mu.Unlock()
if remaining == 0 {
- w.target.OnZeroWatches()
+ w.target.OnZeroWatches(ctx)
}
// Generate the event for the removal.
@@ -462,7 +462,7 @@ func (w *Watches) Remove(id uint64) {
// Notify queues a new event with watches in this set. Watches with
// IN_EXCL_UNLINK are skipped if the event is coming from a child that has been
// unlinked.
-func (w *Watches) Notify(name string, events, cookie uint32, et EventType, unlinked bool) {
+func (w *Watches) Notify(ctx context.Context, name string, events, cookie uint32, et EventType, unlinked bool) {
var hasExpired bool
w.mu.RLock()
for _, watch := range w.ws {
@@ -476,13 +476,13 @@ func (w *Watches) Notify(name string, events, cookie uint32, et EventType, unlin
w.mu.RUnlock()
if hasExpired {
- w.cleanupExpiredWatches()
+ w.cleanupExpiredWatches(ctx)
}
}
// This function is relatively expensive and should only be called where there
// are expired watches.
-func (w *Watches) cleanupExpiredWatches() {
+func (w *Watches) cleanupExpiredWatches(ctx context.Context) {
// Because of lock ordering, we cannot acquire Inotify.mu for each watch
// owner while holding w.mu. As a result, store expired watches locally
// before removing.
@@ -495,15 +495,15 @@ func (w *Watches) cleanupExpiredWatches() {
}
w.mu.RUnlock()
for _, watch := range toRemove {
- watch.owner.RmWatch(watch.wd)
+ watch.owner.RmWatch(ctx, watch.wd)
}
}
// HandleDeletion is called when the watch target is destroyed. Clear the
// watch set, detach watches from the inotify instances they belong to, and
// generate the appropriate events.
-func (w *Watches) HandleDeletion() {
- w.Notify("", linux.IN_DELETE_SELF, 0, InodeEvent, true /* unlinked */)
+func (w *Watches) HandleDeletion(ctx context.Context) {
+ w.Notify(ctx, "", linux.IN_DELETE_SELF, 0, InodeEvent, true /* unlinked */)
// As in Watches.Notify, we can't hold w.mu while acquiring Inotify.mu for
// the owner of each watch being deleted. Instead, atomically store the
@@ -744,12 +744,12 @@ func InotifyEventFromStatMask(mask uint32) uint32 {
// InotifyRemoveChild sends the appriopriate notifications to the watch sets of
// the child being removed and its parent. Note that unlike most pairs of
// parent/child notifications, the child is notified first in this case.
-func InotifyRemoveChild(self, parent *Watches, name string) {
+func InotifyRemoveChild(ctx context.Context, self, parent *Watches, name string) {
if self != nil {
- self.Notify("", linux.IN_ATTRIB, 0, InodeEvent, true /* unlinked */)
+ self.Notify(ctx, "", linux.IN_ATTRIB, 0, InodeEvent, true /* unlinked */)
}
if parent != nil {
- parent.Notify(name, linux.IN_DELETE, 0, InodeEvent, true /* unlinked */)
+ parent.Notify(ctx, name, linux.IN_DELETE, 0, InodeEvent, true /* unlinked */)
}
}
@@ -762,13 +762,13 @@ func InotifyRename(ctx context.Context, renamed, oldParent, newParent *Watches,
}
cookie := uniqueid.InotifyCookie(ctx)
if oldParent != nil {
- oldParent.Notify(oldName, dirEv|linux.IN_MOVED_FROM, cookie, InodeEvent, false /* unlinked */)
+ oldParent.Notify(ctx, oldName, dirEv|linux.IN_MOVED_FROM, cookie, InodeEvent, false /* unlinked */)
}
if newParent != nil {
- newParent.Notify(newName, dirEv|linux.IN_MOVED_TO, cookie, InodeEvent, false /* unlinked */)
+ newParent.Notify(ctx, newName, dirEv|linux.IN_MOVED_TO, cookie, InodeEvent, false /* unlinked */)
}
// Somewhat surprisingly, self move events do not have a cookie.
if renamed != nil {
- renamed.Notify("", linux.IN_MOVE_SELF, 0, InodeEvent, false /* unlinked */)
+ renamed.Notify(ctx, "", linux.IN_MOVE_SELF, 0, InodeEvent, false /* unlinked */)
}
}
diff --git a/pkg/sentry/vfs/mount.go b/pkg/sentry/vfs/mount.go
index 32f901bd8..d1d29d0cd 100644
--- a/pkg/sentry/vfs/mount.go
+++ b/pkg/sentry/vfs/mount.go
@@ -200,8 +200,8 @@ func (vfs *VirtualFilesystem) MountDisconnected(ctx context.Context, creds *auth
if err != nil {
return nil, err
}
- defer root.DecRef()
- defer fs.DecRef()
+ defer root.DecRef(ctx)
+ defer fs.DecRef(ctx)
return vfs.NewDisconnectedMount(fs, root, opts)
}
@@ -221,7 +221,7 @@ func (vfs *VirtualFilesystem) ConnectMountAt(ctx context.Context, creds *auth.Cr
if vd.dentry.dead {
vd.dentry.mu.Unlock()
vfs.mountMu.Unlock()
- vd.DecRef()
+ vd.DecRef(ctx)
return syserror.ENOENT
}
// vd might have been mounted over between vfs.GetDentryAt() and
@@ -243,7 +243,7 @@ func (vfs *VirtualFilesystem) ConnectMountAt(ctx context.Context, creds *auth.Cr
// This can't fail since we're holding vfs.mountMu.
nextmnt.root.IncRef()
vd.dentry.mu.Unlock()
- vd.DecRef()
+ vd.DecRef(ctx)
vd = VirtualDentry{
mount: nextmnt,
dentry: nextmnt.root,
@@ -268,7 +268,7 @@ func (vfs *VirtualFilesystem) MountAt(ctx context.Context, creds *auth.Credentia
if err != nil {
return err
}
- defer mnt.DecRef()
+ defer mnt.DecRef(ctx)
if err := vfs.ConnectMountAt(ctx, creds, mnt, target); err != nil {
return err
}
@@ -293,13 +293,13 @@ func (vfs *VirtualFilesystem) UmountAt(ctx context.Context, creds *auth.Credenti
if err != nil {
return err
}
- defer vd.DecRef()
+ defer vd.DecRef(ctx)
if vd.dentry != vd.mount.root {
return syserror.EINVAL
}
vfs.mountMu.Lock()
if mntns := MountNamespaceFromContext(ctx); mntns != nil {
- defer mntns.DecRef()
+ defer mntns.DecRef(ctx)
if mntns != vd.mount.ns {
vfs.mountMu.Unlock()
return syserror.EINVAL
@@ -335,10 +335,10 @@ func (vfs *VirtualFilesystem) UmountAt(ctx context.Context, creds *auth.Credenti
vfs.mounts.seq.EndWrite()
vfs.mountMu.Unlock()
for _, vd := range vdsToDecRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
for _, mnt := range mountsToDecRef {
- mnt.DecRef()
+ mnt.DecRef(ctx)
}
return nil
}
@@ -479,7 +479,7 @@ func (mnt *Mount) IncRef() {
}
// DecRef decrements mnt's reference count.
-func (mnt *Mount) DecRef() {
+func (mnt *Mount) DecRef(ctx context.Context) {
refs := atomic.AddInt64(&mnt.refs, -1)
if refs&^math.MinInt64 == 0 { // mask out MSB
var vd VirtualDentry
@@ -490,10 +490,10 @@ func (mnt *Mount) DecRef() {
mnt.vfs.mounts.seq.EndWrite()
mnt.vfs.mountMu.Unlock()
}
- mnt.root.DecRef()
- mnt.fs.DecRef()
+ mnt.root.DecRef(ctx)
+ mnt.fs.DecRef(ctx)
if vd.Ok() {
- vd.DecRef()
+ vd.DecRef(ctx)
}
}
}
@@ -506,7 +506,7 @@ func (mntns *MountNamespace) IncRef() {
}
// DecRef decrements mntns' reference count.
-func (mntns *MountNamespace) DecRef() {
+func (mntns *MountNamespace) DecRef(ctx context.Context) {
vfs := mntns.root.fs.VirtualFilesystem()
if refs := atomic.AddInt64(&mntns.refs, -1); refs == 0 {
vfs.mountMu.Lock()
@@ -517,10 +517,10 @@ func (mntns *MountNamespace) DecRef() {
vfs.mounts.seq.EndWrite()
vfs.mountMu.Unlock()
for _, vd := range vdsToDecRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
for _, mnt := range mountsToDecRef {
- mnt.DecRef()
+ mnt.DecRef(ctx)
}
} else if refs < 0 {
panic("MountNamespace.DecRef() called without holding a reference")
@@ -534,7 +534,7 @@ func (mntns *MountNamespace) DecRef() {
// getMountAt is analogous to Linux's fs/namei.c:follow_mount().
//
// Preconditions: References are held on mnt and d.
-func (vfs *VirtualFilesystem) getMountAt(mnt *Mount, d *Dentry) *Mount {
+func (vfs *VirtualFilesystem) getMountAt(ctx context.Context, mnt *Mount, d *Dentry) *Mount {
// The first mount is special-cased:
//
// - The caller is assumed to have checked d.isMounted() already. (This
@@ -565,7 +565,7 @@ retryFirst:
// Raced with umount.
continue
}
- mnt.DecRef()
+ mnt.DecRef(ctx)
mnt = next
d = next.root
}
@@ -578,7 +578,7 @@ retryFirst:
//
// Preconditions: References are held on mnt and root. vfsroot is not (mnt,
// mnt.root).
-func (vfs *VirtualFilesystem) getMountpointAt(mnt *Mount, vfsroot VirtualDentry) VirtualDentry {
+func (vfs *VirtualFilesystem) getMountpointAt(ctx context.Context, mnt *Mount, vfsroot VirtualDentry) VirtualDentry {
// The first mount is special-cased:
//
// - The caller must have already checked mnt against vfsroot.
@@ -602,12 +602,12 @@ retryFirst:
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(ctx)
goto retryFirst
}
if !vfs.mounts.seq.ReadOk(epoch) {
- point.DecRef()
- parent.DecRef()
+ point.DecRef(ctx)
+ parent.DecRef(ctx)
goto retryFirst
}
mnt = parent
@@ -635,16 +635,16 @@ retryFirst:
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(ctx)
goto retryNotFirst
}
if !vfs.mounts.seq.ReadOk(epoch) {
- point.DecRef()
- parent.DecRef()
+ point.DecRef(ctx)
+ parent.DecRef(ctx)
goto retryNotFirst
}
- d.DecRef()
- mnt.DecRef()
+ d.DecRef(ctx)
+ mnt.DecRef(ctx)
mnt = parent
d = point
}
diff --git a/pkg/sentry/vfs/pathname.go b/pkg/sentry/vfs/pathname.go
index cd78d66bc..e4da15009 100644
--- a/pkg/sentry/vfs/pathname.go
+++ b/pkg/sentry/vfs/pathname.go
@@ -47,7 +47,7 @@ func (vfs *VirtualFilesystem) PathnameWithDeleted(ctx context.Context, vfsroot,
haveRef := false
defer func() {
if haveRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
}()
@@ -64,12 +64,12 @@ loop:
// of FilesystemImpl.PrependPath() may return nil instead.
break loop
}
- nextVD := vfs.getMountpointAt(vd.mount, vfsroot)
+ nextVD := vfs.getMountpointAt(ctx, vd.mount, vfsroot)
if !nextVD.Ok() {
break loop
}
if haveRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
vd = nextVD
haveRef = true
@@ -101,7 +101,7 @@ func (vfs *VirtualFilesystem) PathnameReachable(ctx context.Context, vfsroot, vd
haveRef := false
defer func() {
if haveRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
}()
loop:
@@ -112,12 +112,12 @@ loop:
if vd.mount == vfsroot.mount && vd.mount.root == vfsroot.dentry {
break loop
}
- nextVD := vfs.getMountpointAt(vd.mount, vfsroot)
+ nextVD := vfs.getMountpointAt(ctx, vd.mount, vfsroot)
if !nextVD.Ok() {
return "", nil
}
if haveRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
vd = nextVD
haveRef = true
@@ -145,7 +145,7 @@ func (vfs *VirtualFilesystem) PathnameForGetcwd(ctx context.Context, vfsroot, vd
haveRef := false
defer func() {
if haveRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
}()
unreachable := false
@@ -157,13 +157,13 @@ loop:
if vd.mount == vfsroot.mount && vd.mount.root == vfsroot.dentry {
break loop
}
- nextVD := vfs.getMountpointAt(vd.mount, vfsroot)
+ nextVD := vfs.getMountpointAt(ctx, vd.mount, vfsroot)
if !nextVD.Ok() {
unreachable = true
break loop
}
if haveRef {
- vd.DecRef()
+ vd.DecRef(ctx)
}
vd = nextVD
haveRef = true
diff --git a/pkg/sentry/vfs/resolving_path.go b/pkg/sentry/vfs/resolving_path.go
index 9d047ff88..3304372d9 100644
--- a/pkg/sentry/vfs/resolving_path.go
+++ b/pkg/sentry/vfs/resolving_path.go
@@ -18,6 +18,7 @@ import (
"fmt"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sync"
@@ -136,31 +137,31 @@ func (vfs *VirtualFilesystem) getResolvingPath(creds *auth.Credentials, pop *Pat
return rp
}
-func (vfs *VirtualFilesystem) putResolvingPath(rp *ResolvingPath) {
+func (vfs *VirtualFilesystem) putResolvingPath(ctx context.Context, rp *ResolvingPath) {
rp.root = VirtualDentry{}
- rp.decRefStartAndMount()
+ rp.decRefStartAndMount(ctx)
rp.mount = nil
rp.start = nil
- rp.releaseErrorState()
+ rp.releaseErrorState(ctx)
resolvingPathPool.Put(rp)
}
-func (rp *ResolvingPath) decRefStartAndMount() {
+func (rp *ResolvingPath) decRefStartAndMount(ctx context.Context) {
if rp.flags&rpflagsHaveStartRef != 0 {
- rp.start.DecRef()
+ rp.start.DecRef(ctx)
}
if rp.flags&rpflagsHaveMountRef != 0 {
- rp.mount.DecRef()
+ rp.mount.DecRef(ctx)
}
}
-func (rp *ResolvingPath) releaseErrorState() {
+func (rp *ResolvingPath) releaseErrorState(ctx context.Context) {
if rp.nextStart != nil {
- rp.nextStart.DecRef()
+ rp.nextStart.DecRef(ctx)
rp.nextStart = nil
}
if rp.nextMount != nil {
- rp.nextMount.DecRef()
+ rp.nextMount.DecRef(ctx)
rp.nextMount = nil
}
}
@@ -236,13 +237,13 @@ func (rp *ResolvingPath) Advance() {
// Restart resets the stream of path components represented by rp to its state
// on entry to the current FilesystemImpl method.
-func (rp *ResolvingPath) Restart() {
+func (rp *ResolvingPath) Restart(ctx context.Context) {
rp.pit = rp.origParts[rp.numOrigParts-1]
rp.mustBeDir = rp.mustBeDirOrig
rp.symlinks = rp.symlinksOrig
rp.curPart = rp.numOrigParts - 1
copy(rp.parts[:], rp.origParts[:rp.numOrigParts])
- rp.releaseErrorState()
+ rp.releaseErrorState(ctx)
}
func (rp *ResolvingPath) relpathCommit() {
@@ -260,13 +261,13 @@ func (rp *ResolvingPath) relpathCommit() {
// Mount, CheckRoot returns (unspecified, non-nil error). Otherwise, path
// resolution should resolve d's parent normally, and CheckRoot returns (false,
// nil).
-func (rp *ResolvingPath) CheckRoot(d *Dentry) (bool, error) {
+func (rp *ResolvingPath) CheckRoot(ctx context.Context, d *Dentry) (bool, error) {
if d == rp.root.dentry && rp.mount == rp.root.mount {
// At contextual VFS root (due to e.g. chroot(2)).
return true, nil
} else if d == rp.mount.root {
// At mount root ...
- vd := rp.vfs.getMountpointAt(rp.mount, rp.root)
+ vd := rp.vfs.getMountpointAt(ctx, rp.mount, rp.root)
if vd.Ok() {
// ... of non-root mount.
rp.nextMount = vd.mount
@@ -283,11 +284,11 @@ func (rp *ResolvingPath) CheckRoot(d *Dentry) (bool, error) {
// to d. If d is a mount point, such that path resolution should switch to
// another Mount, CheckMount returns a non-nil error. Otherwise, CheckMount
// returns nil.
-func (rp *ResolvingPath) CheckMount(d *Dentry) error {
+func (rp *ResolvingPath) CheckMount(ctx context.Context, d *Dentry) error {
if !d.isMounted() {
return nil
}
- if mnt := rp.vfs.getMountAt(rp.mount, d); mnt != nil {
+ if mnt := rp.vfs.getMountAt(ctx, rp.mount, d); mnt != nil {
rp.nextMount = mnt
return resolveMountPointError{}
}
@@ -389,11 +390,11 @@ func (rp *ResolvingPath) HandleJump(target VirtualDentry) error {
return resolveMountRootOrJumpError{}
}
-func (rp *ResolvingPath) handleError(err error) bool {
+func (rp *ResolvingPath) handleError(ctx context.Context, err error) bool {
switch err.(type) {
case resolveMountRootOrJumpError:
// Switch to the new Mount. We hold references on the Mount and Dentry.
- rp.decRefStartAndMount()
+ rp.decRefStartAndMount(ctx)
rp.mount = rp.nextMount
rp.start = rp.nextStart
rp.flags |= rpflagsHaveMountRef | rpflagsHaveStartRef
@@ -412,7 +413,7 @@ func (rp *ResolvingPath) handleError(err error) bool {
case resolveMountPointError:
// Switch to the new Mount. We hold a reference on the Mount, but
// borrow the reference on the mount root from the Mount.
- rp.decRefStartAndMount()
+ rp.decRefStartAndMount(ctx)
rp.mount = rp.nextMount
rp.start = rp.nextMount.root
rp.flags = rp.flags&^rpflagsHaveStartRef | rpflagsHaveMountRef
@@ -423,12 +424,12 @@ func (rp *ResolvingPath) handleError(err error) bool {
// path.
rp.relpathCommit()
// Restart path resolution on the new Mount.
- rp.releaseErrorState()
+ rp.releaseErrorState(ctx)
return true
case resolveAbsSymlinkError:
// Switch to the new Mount. References are borrowed from rp.root.
- rp.decRefStartAndMount()
+ rp.decRefStartAndMount(ctx)
rp.mount = rp.root.mount
rp.start = rp.root.dentry
rp.flags &^= rpflagsHaveMountRef | rpflagsHaveStartRef
@@ -440,7 +441,7 @@ func (rp *ResolvingPath) handleError(err error) bool {
// path, including the symlink target we just prepended.
rp.relpathCommit()
// Restart path resolution on the new Mount.
- rp.releaseErrorState()
+ rp.releaseErrorState(ctx)
return true
default:
diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go
index 522e27475..9c2420683 100644
--- a/pkg/sentry/vfs/vfs.go
+++ b/pkg/sentry/vfs/vfs.go
@@ -122,7 +122,7 @@ type VirtualFilesystem struct {
}
// Init initializes a new VirtualFilesystem with no mounts or FilesystemTypes.
-func (vfs *VirtualFilesystem) Init() error {
+func (vfs *VirtualFilesystem) Init(ctx context.Context) error {
if vfs.mountpoints != nil {
panic("VFS already initialized")
}
@@ -145,7 +145,7 @@ func (vfs *VirtualFilesystem) Init() error {
devMinor: anonfsDevMinor,
}
anonfs.vfsfs.Init(vfs, &anonFilesystemType{}, &anonfs)
- defer anonfs.vfsfs.DecRef()
+ defer anonfs.vfsfs.DecRef(ctx)
anonMount, err := vfs.NewDisconnectedMount(&anonfs.vfsfs, nil, &MountOptions{})
if err != nil {
// We should not be passing any MountOptions that would cause
@@ -192,11 +192,11 @@ func (vfs *VirtualFilesystem) AccessAt(ctx context.Context, creds *auth.Credenti
for {
err := rp.mount.fs.impl.AccessAt(ctx, rp, creds, ats)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -214,11 +214,11 @@ func (vfs *VirtualFilesystem) GetDentryAt(ctx context.Context, creds *auth.Crede
dentry: d,
}
rp.mount.IncRef()
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return vd, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return VirtualDentry{}, err
}
}
@@ -236,7 +236,7 @@ func (vfs *VirtualFilesystem) getParentDirAndName(ctx context.Context, creds *au
}
rp.mount.IncRef()
name := rp.Component()
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return parentVD, name, nil
}
if checkInvariants {
@@ -244,8 +244,8 @@ func (vfs *VirtualFilesystem) getParentDirAndName(ctx context.Context, creds *au
panic(fmt.Sprintf("%T.GetParentDentryAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return VirtualDentry{}, "", err
}
}
@@ -260,14 +260,14 @@ func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credential
}
if !newpop.Path.Begin.Ok() {
- oldVD.DecRef()
+ oldVD.DecRef(ctx)
if newpop.Path.Absolute {
return syserror.EEXIST
}
return syserror.ENOENT
}
if newpop.FollowFinalSymlink {
- oldVD.DecRef()
+ oldVD.DecRef(ctx)
ctx.Warningf("VirtualFilesystem.LinkAt: file creation paths can't follow final symlink")
return syserror.EINVAL
}
@@ -276,8 +276,8 @@ func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credential
for {
err := rp.mount.fs.impl.LinkAt(ctx, rp, oldVD)
if err == nil {
- vfs.putResolvingPath(rp)
- oldVD.DecRef()
+ vfs.putResolvingPath(ctx, rp)
+ oldVD.DecRef(ctx)
return nil
}
if checkInvariants {
@@ -285,9 +285,9 @@ func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credential
panic(fmt.Sprintf("%T.LinkAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- oldVD.DecRef()
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
+ oldVD.DecRef(ctx)
return err
}
}
@@ -313,7 +313,7 @@ func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentia
for {
err := rp.mount.fs.impl.MkdirAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
if checkInvariants {
@@ -321,8 +321,8 @@ func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentia
panic(fmt.Sprintf("%T.MkdirAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -346,7 +346,7 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia
for {
err := rp.mount.fs.impl.MknodAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
if checkInvariants {
@@ -354,8 +354,8 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia
panic(fmt.Sprintf("%T.MknodAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -408,31 +408,31 @@ func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credential
for {
fd, err := rp.mount.fs.impl.OpenAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
if opts.FileExec {
if fd.Mount().Flags.NoExec {
- fd.DecRef()
+ fd.DecRef(ctx)
return nil, syserror.EACCES
}
// Only a regular file can be executed.
stat, err := fd.Stat(ctx, StatOptions{Mask: linux.STATX_TYPE})
if err != nil {
- fd.DecRef()
+ fd.DecRef(ctx)
return nil, err
}
if stat.Mask&linux.STATX_TYPE == 0 || stat.Mode&linux.S_IFMT != linux.S_IFREG {
- fd.DecRef()
+ fd.DecRef(ctx)
return nil, syserror.EACCES
}
}
- fd.Dentry().InotifyWithParent(linux.IN_OPEN, 0, PathEvent)
+ fd.Dentry().InotifyWithParent(ctx, linux.IN_OPEN, 0, PathEvent)
return fd, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return nil, err
}
}
@@ -444,11 +444,11 @@ func (vfs *VirtualFilesystem) ReadlinkAt(ctx context.Context, creds *auth.Creden
for {
target, err := rp.mount.fs.impl.ReadlinkAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return target, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return "", err
}
}
@@ -472,19 +472,19 @@ func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credenti
return err
}
if oldName == "." || oldName == ".." {
- oldParentVD.DecRef()
+ oldParentVD.DecRef(ctx)
return syserror.EBUSY
}
if !newpop.Path.Begin.Ok() {
- oldParentVD.DecRef()
+ oldParentVD.DecRef(ctx)
if newpop.Path.Absolute {
return syserror.EBUSY
}
return syserror.ENOENT
}
if newpop.FollowFinalSymlink {
- oldParentVD.DecRef()
+ oldParentVD.DecRef(ctx)
ctx.Warningf("VirtualFilesystem.RenameAt: destination path can't follow final symlink")
return syserror.EINVAL
}
@@ -497,8 +497,8 @@ func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credenti
for {
err := rp.mount.fs.impl.RenameAt(ctx, rp, oldParentVD, oldName, renameOpts)
if err == nil {
- vfs.putResolvingPath(rp)
- oldParentVD.DecRef()
+ vfs.putResolvingPath(ctx, rp)
+ oldParentVD.DecRef(ctx)
return nil
}
if checkInvariants {
@@ -506,9 +506,9 @@ func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credenti
panic(fmt.Sprintf("%T.RenameAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- oldParentVD.DecRef()
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
+ oldParentVD.DecRef(ctx)
return err
}
}
@@ -531,7 +531,7 @@ func (vfs *VirtualFilesystem) RmdirAt(ctx context.Context, creds *auth.Credentia
for {
err := rp.mount.fs.impl.RmdirAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
if checkInvariants {
@@ -539,8 +539,8 @@ func (vfs *VirtualFilesystem) RmdirAt(ctx context.Context, creds *auth.Credentia
panic(fmt.Sprintf("%T.RmdirAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -552,11 +552,11 @@ func (vfs *VirtualFilesystem) SetStatAt(ctx context.Context, creds *auth.Credent
for {
err := rp.mount.fs.impl.SetStatAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -568,11 +568,11 @@ func (vfs *VirtualFilesystem) StatAt(ctx context.Context, creds *auth.Credential
for {
stat, err := rp.mount.fs.impl.StatAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return stat, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return linux.Statx{}, err
}
}
@@ -585,11 +585,11 @@ func (vfs *VirtualFilesystem) StatFSAt(ctx context.Context, creds *auth.Credenti
for {
statfs, err := rp.mount.fs.impl.StatFSAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return statfs, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return linux.Statfs{}, err
}
}
@@ -612,7 +612,7 @@ func (vfs *VirtualFilesystem) SymlinkAt(ctx context.Context, creds *auth.Credent
for {
err := rp.mount.fs.impl.SymlinkAt(ctx, rp, target)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
if checkInvariants {
@@ -620,8 +620,8 @@ func (vfs *VirtualFilesystem) SymlinkAt(ctx context.Context, creds *auth.Credent
panic(fmt.Sprintf("%T.SymlinkAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -644,7 +644,7 @@ func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credenti
for {
err := rp.mount.fs.impl.UnlinkAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
if checkInvariants {
@@ -652,8 +652,8 @@ func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credenti
panic(fmt.Sprintf("%T.UnlinkAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -671,7 +671,7 @@ func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.C
for {
bep, err := rp.mount.fs.impl.BoundEndpointAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return bep, nil
}
if checkInvariants {
@@ -679,8 +679,8 @@ func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.C
panic(fmt.Sprintf("%T.BoundEndpointAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return nil, err
}
}
@@ -693,7 +693,7 @@ func (vfs *VirtualFilesystem) ListxattrAt(ctx context.Context, creds *auth.Crede
for {
names, err := rp.mount.fs.impl.ListxattrAt(ctx, rp, size)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return names, nil
}
if err == syserror.ENOTSUP {
@@ -701,11 +701,11 @@ func (vfs *VirtualFilesystem) ListxattrAt(ctx context.Context, creds *auth.Crede
// fs/xattr.c:vfs_listxattr() falls back to allowing the security
// subsystem to return security extended attributes, which by
// default don't exist.
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return nil, err
}
}
@@ -718,11 +718,11 @@ func (vfs *VirtualFilesystem) GetxattrAt(ctx context.Context, creds *auth.Creden
for {
val, err := rp.mount.fs.impl.GetxattrAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return val, nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return "", err
}
}
@@ -735,11 +735,11 @@ func (vfs *VirtualFilesystem) SetxattrAt(ctx context.Context, creds *auth.Creden
for {
err := rp.mount.fs.impl.SetxattrAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -751,11 +751,11 @@ func (vfs *VirtualFilesystem) RemovexattrAt(ctx context.Context, creds *auth.Cre
for {
err := rp.mount.fs.impl.RemovexattrAt(ctx, rp, name)
if err == nil {
- vfs.putResolvingPath(rp)
+ vfs.putResolvingPath(ctx, rp)
return nil
}
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
+ if !rp.handleError(ctx, err) {
+ vfs.putResolvingPath(ctx, rp)
return err
}
}
@@ -777,7 +777,7 @@ func (vfs *VirtualFilesystem) SyncAllFilesystems(ctx context.Context) error {
if err := fs.impl.Sync(ctx); err != nil && retErr == nil {
retErr = err
}
- fs.DecRef()
+ fs.DecRef(ctx)
}
return retErr
}
@@ -831,9 +831,9 @@ func (vd VirtualDentry) IncRef() {
// DecRef decrements the reference counts on the Mount and Dentry represented
// by vd.
-func (vd VirtualDentry) DecRef() {
- vd.dentry.DecRef()
- vd.mount.DecRef()
+func (vd VirtualDentry) DecRef(ctx context.Context) {
+ vd.dentry.DecRef(ctx)
+ vd.mount.DecRef(ctx)
}
// Mount returns the Mount associated with vd. It does not take a reference on