summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/vfs
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/vfs')
-rw-r--r--pkg/sentry/vfs/BUILD2
-rw-r--r--pkg/sentry/vfs/context.go13
-rw-r--r--pkg/sentry/vfs/file_description.go137
-rw-r--r--pkg/sentry/vfs/file_description_impl_util.go9
-rw-r--r--pkg/sentry/vfs/file_description_impl_util_test.go10
-rw-r--r--pkg/sentry/vfs/filesystem.go76
-rw-r--r--pkg/sentry/vfs/filesystem_impl_util.go26
-rw-r--r--pkg/sentry/vfs/mount.go69
-rw-r--r--pkg/sentry/vfs/options.go12
-rw-r--r--pkg/sentry/vfs/pathname.go153
-rw-r--r--pkg/sentry/vfs/permissions.go62
-rw-r--r--pkg/sentry/vfs/syscalls.go237
-rw-r--r--pkg/sentry/vfs/testutil.go9
-rw-r--r--pkg/sentry/vfs/vfs.go378
14 files changed, 946 insertions, 247 deletions
diff --git a/pkg/sentry/vfs/BUILD b/pkg/sentry/vfs/BUILD
index 74a325309..e3e554b88 100644
--- a/pkg/sentry/vfs/BUILD
+++ b/pkg/sentry/vfs/BUILD
@@ -17,9 +17,9 @@ go_library(
"mount.go",
"mount_unsafe.go",
"options.go",
+ "pathname.go",
"permissions.go",
"resolving_path.go",
- "syscalls.go",
"testutil.go",
"vfs.go",
],
diff --git a/pkg/sentry/vfs/context.go b/pkg/sentry/vfs/context.go
index 32cf9151b..705194ebc 100644
--- a/pkg/sentry/vfs/context.go
+++ b/pkg/sentry/vfs/context.go
@@ -24,6 +24,9 @@ type contextID int
const (
// CtxMountNamespace is a Context.Value key for a MountNamespace.
CtxMountNamespace contextID = iota
+
+ // CtxRoot is a Context.Value key for a VFS root.
+ CtxRoot
)
// MountNamespaceFromContext returns the MountNamespace used by ctx. It does
@@ -35,3 +38,13 @@ func MountNamespaceFromContext(ctx context.Context) *MountNamespace {
}
return nil
}
+
+// RootFromContext returns the VFS root used by ctx. It takes a reference on
+// the returned VirtualDentry. If ctx does not have a specific VFS root,
+// RootFromContext returns a zero-value VirtualDentry.
+func RootFromContext(ctx context.Context) VirtualDentry {
+ if v := ctx.Value(CtxRoot); v != nil {
+ return v.(VirtualDentry)
+ }
+ return VirtualDentry{}
+}
diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go
index 34007eb57..6575afd16 100644
--- a/pkg/sentry/vfs/file_description.go
+++ b/pkg/sentry/vfs/file_description.go
@@ -241,3 +241,140 @@ type IterDirentsCallback interface {
// called.
Handle(dirent Dirent) bool
}
+
+// OnClose is called when a file descriptor representing the FileDescription is
+// closed. Returning a non-nil error should not prevent the file descriptor
+// from being closed.
+func (fd *FileDescription) OnClose(ctx context.Context) error {
+ return fd.impl.OnClose(ctx)
+}
+
+// StatusFlags returns file description status flags, as for fcntl(F_GETFL).
+func (fd *FileDescription) StatusFlags(ctx context.Context) (uint32, error) {
+ flags, err := fd.impl.StatusFlags(ctx)
+ flags |= linux.O_LARGEFILE
+ return flags, err
+}
+
+// SetStatusFlags sets file description status flags, as for fcntl(F_SETFL).
+func (fd *FileDescription) SetStatusFlags(ctx context.Context, flags uint32) error {
+ return fd.impl.SetStatusFlags(ctx, flags)
+}
+
+// Stat returns metadata for the file represented by fd.
+func (fd *FileDescription) Stat(ctx context.Context, opts StatOptions) (linux.Statx, error) {
+ return fd.impl.Stat(ctx, opts)
+}
+
+// SetStat updates metadata for the file represented by fd.
+func (fd *FileDescription) SetStat(ctx context.Context, opts SetStatOptions) error {
+ return fd.impl.SetStat(ctx, opts)
+}
+
+// StatFS returns metadata for the filesystem containing the file represented
+// by fd.
+func (fd *FileDescription) StatFS(ctx context.Context) (linux.Statfs, error) {
+ return fd.impl.StatFS(ctx)
+}
+
+// PRead reads from the file represented by fd into dst, starting at the given
+// offset, and returns the number of bytes read. PRead is permitted to return
+// partial reads with a nil error.
+func (fd *FileDescription) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts ReadOptions) (int64, error) {
+ return fd.impl.PRead(ctx, dst, offset, opts)
+}
+
+// Read is similar to PRead, but does not specify an offset.
+func (fd *FileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts ReadOptions) (int64, error) {
+ return fd.impl.Read(ctx, dst, opts)
+}
+
+// PWrite writes src to the file represented by fd, starting at the given
+// offset, and returns the number of bytes written. PWrite is permitted to
+// return partial writes with a nil error.
+func (fd *FileDescription) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts WriteOptions) (int64, error) {
+ return fd.impl.PWrite(ctx, src, offset, opts)
+}
+
+// Write is similar to PWrite, but does not specify an offset.
+func (fd *FileDescription) Write(ctx context.Context, src usermem.IOSequence, opts WriteOptions) (int64, error) {
+ return fd.impl.Write(ctx, src, opts)
+}
+
+// IterDirents invokes cb on each entry in the directory represented by fd. If
+// IterDirents has been called since the last call to Seek, it continues
+// iteration from the end of the last call.
+func (fd *FileDescription) IterDirents(ctx context.Context, cb IterDirentsCallback) error {
+ return fd.impl.IterDirents(ctx, cb)
+}
+
+// Seek changes fd's offset (assuming one exists) and returns its new value.
+func (fd *FileDescription) Seek(ctx context.Context, offset int64, whence int32) (int64, error) {
+ return fd.impl.Seek(ctx, offset, whence)
+}
+
+// Sync has the semantics of fsync(2).
+func (fd *FileDescription) Sync(ctx context.Context) error {
+ return fd.impl.Sync(ctx)
+}
+
+// ConfigureMMap mutates opts to implement mmap(2) for the file represented by
+// fd.
+func (fd *FileDescription) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error {
+ return fd.impl.ConfigureMMap(ctx, opts)
+}
+
+// Ioctl implements the ioctl(2) syscall.
+func (fd *FileDescription) Ioctl(ctx context.Context, uio usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ return fd.impl.Ioctl(ctx, uio, args)
+}
+
+// SyncFS instructs the filesystem containing fd to execute the semantics of
+// syncfs(2).
+func (fd *FileDescription) SyncFS(ctx context.Context) error {
+ return fd.vd.mount.fs.impl.Sync(ctx)
+}
+
+// MappedName implements memmap.MappingIdentity.MappedName.
+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()
+ }
+ return s
+}
+
+// DeviceID implements memmap.MappingIdentity.DeviceID.
+func (fd *FileDescription) DeviceID() uint64 {
+ stat, err := fd.impl.Stat(context.Background(), StatOptions{
+ // There is no STATX_DEV; we assume that Stat will return it if it's
+ // available regardless of mask.
+ Mask: 0,
+ // fs/proc/task_mmu.c:show_map_vma() just reads inode::i_sb->s_dev
+ // directly.
+ Sync: linux.AT_STATX_DONT_SYNC,
+ })
+ if err != nil {
+ return 0
+ }
+ return uint64(linux.MakeDeviceID(uint16(stat.DevMajor), stat.DevMinor))
+}
+
+// InodeID implements memmap.MappingIdentity.InodeID.
+func (fd *FileDescription) InodeID() uint64 {
+ stat, err := fd.impl.Stat(context.Background(), StatOptions{
+ Mask: linux.STATX_INO,
+ // fs/proc/task_mmu.c:show_map_vma() just reads inode::i_ino directly.
+ Sync: linux.AT_STATX_DONT_SYNC,
+ })
+ if err != nil || stat.Mask&linux.STATX_INO == 0 {
+ return 0
+ }
+ return stat.Ino
+}
+
+// Msync implements memmap.MappingIdentity.Msync.
+func (fd *FileDescription) Msync(ctx context.Context, mr memmap.MappableRange) error {
+ return fd.impl.Sync(ctx)
+}
diff --git a/pkg/sentry/vfs/file_description_impl_util.go b/pkg/sentry/vfs/file_description_impl_util.go
index 4fbad7840..aae023254 100644
--- a/pkg/sentry/vfs/file_description_impl_util.go
+++ b/pkg/sentry/vfs/file_description_impl_util.go
@@ -252,3 +252,12 @@ func (fd *DynamicBytesFileDescriptionImpl) Seek(ctx context.Context, offset int6
fd.off = offset
return offset, nil
}
+
+// GenericConfigureMMap may be used by most implementations of
+// FileDescriptionImpl.ConfigureMMap.
+func GenericConfigureMMap(fd *FileDescription, m memmap.Mappable, opts *memmap.MMapOpts) error {
+ opts.Mappable = m
+ opts.MappingIdentity = fd
+ fd.IncRef()
+ return nil
+}
diff --git a/pkg/sentry/vfs/file_description_impl_util_test.go b/pkg/sentry/vfs/file_description_impl_util_test.go
index a5561dcbe..ac7799296 100644
--- a/pkg/sentry/vfs/file_description_impl_util_test.go
+++ b/pkg/sentry/vfs/file_description_impl_util_test.go
@@ -103,7 +103,7 @@ func TestGenCountFD(t *testing.T) {
// The first read causes Generate to be called to fill the FD's buffer.
buf := make([]byte, 2)
ioseq := usermem.BytesIOSequence(buf)
- n, err := fd.Impl().Read(ctx, ioseq, ReadOptions{})
+ n, err := fd.Read(ctx, ioseq, ReadOptions{})
if n != 1 || (err != nil && err != io.EOF) {
t.Fatalf("first Read: got (%d, %v), wanted (1, nil or EOF)", n, err)
}
@@ -112,17 +112,17 @@ func TestGenCountFD(t *testing.T) {
}
// A second read without seeking is still at EOF.
- n, err = fd.Impl().Read(ctx, ioseq, ReadOptions{})
+ n, err = fd.Read(ctx, ioseq, ReadOptions{})
if n != 0 || err != io.EOF {
t.Fatalf("second Read: got (%d, %v), wanted (0, EOF)", n, err)
}
// Seeking to the beginning of the file causes it to be regenerated.
- n, err = fd.Impl().Seek(ctx, 0, linux.SEEK_SET)
+ n, err = fd.Seek(ctx, 0, linux.SEEK_SET)
if n != 0 || err != nil {
t.Fatalf("Seek: got (%d, %v), wanted (0, nil)", n, err)
}
- n, err = fd.Impl().Read(ctx, ioseq, ReadOptions{})
+ n, err = fd.Read(ctx, ioseq, ReadOptions{})
if n != 1 || (err != nil && err != io.EOF) {
t.Fatalf("Read after Seek: got (%d, %v), wanted (1, nil or EOF)", n, err)
}
@@ -131,7 +131,7 @@ func TestGenCountFD(t *testing.T) {
}
// PRead at the beginning of the file also causes it to be regenerated.
- n, err = fd.Impl().PRead(ctx, ioseq, 0, ReadOptions{})
+ n, err = fd.PRead(ctx, ioseq, 0, ReadOptions{})
if n != 1 || (err != nil && err != io.EOF) {
t.Fatalf("PRead: got (%d, %v), wanted (1, nil or EOF)", n, err)
}
diff --git a/pkg/sentry/vfs/filesystem.go b/pkg/sentry/vfs/filesystem.go
index 76ff8cf51..8011eba3f 100644
--- a/pkg/sentry/vfs/filesystem.go
+++ b/pkg/sentry/vfs/filesystem.go
@@ -18,6 +18,7 @@ import (
"sync/atomic"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/context"
)
@@ -47,6 +48,9 @@ func (fs *Filesystem) Init(vfsObj *VirtualFilesystem, impl FilesystemImpl) {
fs.refs = 1
fs.vfs = vfsObj
fs.impl = impl
+ vfsObj.filesystemsMu.Lock()
+ vfsObj.filesystems[fs] = struct{}{}
+ vfsObj.filesystemsMu.Unlock()
}
// VirtualFilesystem returns the containing VirtualFilesystem.
@@ -66,9 +70,28 @@ func (fs *Filesystem) IncRef() {
}
}
+// TryIncRef increments fs' reference count and returns true. If fs' reference
+// count is zero, TryIncRef does nothing and returns false.
+//
+// TryIncRef does not require that a reference is held on fs.
+func (fs *Filesystem) TryIncRef() bool {
+ for {
+ refs := atomic.LoadInt64(&fs.refs)
+ if refs <= 0 {
+ return false
+ }
+ if atomic.CompareAndSwapInt64(&fs.refs, refs, refs+1) {
+ return true
+ }
+ }
+}
+
// DecRef decrements fs' reference count.
func (fs *Filesystem) DecRef() {
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()
} else if refs < 0 {
panic("Filesystem.decRef() called without holding a reference")
@@ -163,5 +186,56 @@ type FilesystemImpl interface {
// UnlinkAt removes the non-directory file at rp.
UnlinkAt(ctx context.Context, rp *ResolvingPath) error
- // TODO: d_path(); extended attributes; inotify_add_watch(); bind()
+ // PrependPath prepends a path from vd to vd.Mount().Root() to b.
+ //
+ // If vfsroot.Ok(), it is the contextual VFS root; if it is encountered
+ // before vd.Mount().Root(), PrependPath should stop prepending path
+ // components and return a PrependPathAtVFSRootError.
+ //
+ // If traversal of vd.Dentry()'s ancestors encounters an independent
+ // ("root") Dentry that is not vd.Mount().Root() (i.e. vd.Dentry() is not a
+ // descendant of vd.Mount().Root()), PrependPath should stop prepending
+ // path components and return a PrependPathAtNonMountRootError.
+ //
+ // Filesystems for which Dentries do not have meaningful paths may prepend
+ // an arbitrary descriptive string to b and then return a
+ // PrependPathSyntheticError.
+ //
+ // Most implementations can acquire the appropriate locks to ensure that
+ // Dentry.Name() and Dentry.Parent() are fixed for vd.Dentry() and all of
+ // its ancestors, then call GenericPrependPath.
+ //
+ // Preconditions: vd.Mount().Filesystem().Impl() == this FilesystemImpl.
+ PrependPath(ctx context.Context, vfsroot, vd VirtualDentry, b *fspath.Builder) error
+
+ // TODO: extended attributes; inotify_add_watch(); bind()
+}
+
+// PrependPathAtVFSRootError is returned by implementations of
+// FilesystemImpl.PrependPath() when they encounter the contextual VFS root.
+type PrependPathAtVFSRootError struct{}
+
+// Error implements error.Error.
+func (PrependPathAtVFSRootError) Error() string {
+ return "vfs.FilesystemImpl.PrependPath() reached VFS root"
+}
+
+// PrependPathAtNonMountRootError is returned by implementations of
+// FilesystemImpl.PrependPath() when they encounter an independent ancestor
+// Dentry that is not the Mount root.
+type PrependPathAtNonMountRootError struct{}
+
+// Error implements error.Error.
+func (PrependPathAtNonMountRootError) Error() string {
+ return "vfs.FilesystemImpl.PrependPath() reached root other than Mount root"
+}
+
+// PrependPathSyntheticError is returned by implementations of
+// FilesystemImpl.PrependPath() for which prepended names do not represent real
+// paths.
+type PrependPathSyntheticError struct{}
+
+// Error implements error.Error.
+func (PrependPathSyntheticError) Error() string {
+ return "vfs.FilesystemImpl.PrependPath() prepended synthetic name"
}
diff --git a/pkg/sentry/vfs/filesystem_impl_util.go b/pkg/sentry/vfs/filesystem_impl_util.go
index 465e610e0..7315a588e 100644
--- a/pkg/sentry/vfs/filesystem_impl_util.go
+++ b/pkg/sentry/vfs/filesystem_impl_util.go
@@ -16,6 +16,8 @@ package vfs
import (
"strings"
+
+ "gvisor.dev/gvisor/pkg/fspath"
)
// GenericParseMountOptions parses a comma-separated list of options of the
@@ -41,3 +43,27 @@ func GenericParseMountOptions(str string) map[string]string {
}
return m
}
+
+// GenericPrependPath may be used by implementations of
+// FilesystemImpl.PrependPath() for which a single statically-determined lock
+// or set of locks is sufficient to ensure its preconditions (as opposed to
+// e.g. per-Dentry locks).
+//
+// Preconditions: Dentry.Name() and Dentry.Parent() must be held constant for
+// vd.Dentry() and all of its ancestors.
+func GenericPrependPath(vfsroot, vd VirtualDentry, b *fspath.Builder) error {
+ mnt, d := vd.mount, vd.dentry
+ for {
+ if mnt == vfsroot.mount && d == vfsroot.dentry {
+ return PrependPathAtVFSRootError{}
+ }
+ if d == mnt.root {
+ return nil
+ }
+ if d.parent == nil {
+ return PrependPathAtNonMountRootError{}
+ }
+ b.PrependComponent(d.name)
+ d = d.parent
+ }
+}
diff --git a/pkg/sentry/vfs/mount.go b/pkg/sentry/vfs/mount.go
index 1c3b2e987..ec23ab0dd 100644
--- a/pkg/sentry/vfs/mount.go
+++ b/pkg/sentry/vfs/mount.go
@@ -18,6 +18,7 @@ import (
"math"
"sync/atomic"
+ "gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/context"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/syserror"
@@ -133,13 +134,13 @@ func (vfs *VirtualFilesystem) NewMountNamespace(ctx context.Context, creds *auth
return mntns, nil
}
-// NewMount creates and mounts a Filesystem configured by the given arguments.
-func (vfs *VirtualFilesystem) NewMount(ctx context.Context, creds *auth.Credentials, source string, target *PathOperation, fsTypeName string, opts *GetFilesystemOptions) error {
+// MountAt creates and mounts a Filesystem configured by the given arguments.
+func (vfs *VirtualFilesystem) MountAt(ctx context.Context, creds *auth.Credentials, source string, target *PathOperation, fsTypeName string, opts *MountOptions) error {
fsType := vfs.getFilesystemType(fsTypeName)
if fsType == nil {
return syserror.ENODEV
}
- fs, root, err := fsType.GetFilesystem(ctx, vfs, creds, source, *opts)
+ fs, root, err := fsType.GetFilesystem(ctx, vfs, creds, source, opts.GetFilesystemOptions)
if err != nil {
return err
}
@@ -207,6 +208,68 @@ func (vfs *VirtualFilesystem) NewMount(ctx context.Context, creds *auth.Credenti
return nil
}
+// UmountAt removes the Mount at the given path.
+func (vfs *VirtualFilesystem) UmountAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *UmountOptions) error {
+ if opts.Flags&^(linux.MNT_FORCE|linux.MNT_DETACH) != 0 {
+ return syserror.EINVAL
+ }
+
+ // MNT_FORCE is currently unimplemented except for the permission check.
+ if opts.Flags&linux.MNT_FORCE != 0 && creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, creds.UserNamespace.Root()) {
+ return syserror.EPERM
+ }
+
+ vd, err := vfs.GetDentryAt(ctx, creds, pop, &GetDentryOptions{})
+ if err != nil {
+ return err
+ }
+ defer vd.DecRef()
+ if vd.dentry != vd.mount.root {
+ return syserror.EINVAL
+ }
+ vfs.mountMu.Lock()
+ if mntns := MountNamespaceFromContext(ctx); mntns != nil && mntns != vd.mount.ns {
+ vfs.mountMu.Unlock()
+ return syserror.EINVAL
+ }
+
+ // TODO(jamieliu): Linux special-cases umount of the caller's root, which
+ // we don't implement yet (we'll just fail it since the caller holds a
+ // reference on it).
+
+ vfs.mounts.seq.BeginWrite()
+ if opts.Flags&linux.MNT_DETACH == 0 {
+ if len(vd.mount.children) != 0 {
+ vfs.mounts.seq.EndWrite()
+ vfs.mountMu.Unlock()
+ return syserror.EBUSY
+ }
+ // We are holding a reference on vd.mount.
+ expectedRefs := int64(1)
+ if !vd.mount.umounted {
+ expectedRefs = 2
+ }
+ if atomic.LoadInt64(&vd.mount.refs)&^math.MinInt64 != expectedRefs { // mask out MSB
+ vfs.mounts.seq.EndWrite()
+ vfs.mountMu.Unlock()
+ return syserror.EBUSY
+ }
+ }
+ vdsToDecRef, mountsToDecRef := vfs.umountRecursiveLocked(vd.mount, &umountRecursiveOptions{
+ eager: opts.Flags&linux.MNT_DETACH == 0,
+ disconnectHierarchy: true,
+ }, nil, nil)
+ vfs.mounts.seq.EndWrite()
+ vfs.mountMu.Unlock()
+ for _, vd := range vdsToDecRef {
+ vd.DecRef()
+ }
+ for _, mnt := range mountsToDecRef {
+ mnt.DecRef()
+ }
+ return nil
+}
+
type umountRecursiveOptions struct {
// If eager is true, ensure that future calls to Mount.tryIncMountedRef()
// on umounted mounts fail.
diff --git a/pkg/sentry/vfs/options.go b/pkg/sentry/vfs/options.go
index 3aa73d911..3ecbc8fc1 100644
--- a/pkg/sentry/vfs/options.go
+++ b/pkg/sentry/vfs/options.go
@@ -46,6 +46,12 @@ type MknodOptions struct {
DevMinor uint32
}
+// MountOptions contains options to VirtualFilesystem.MountAt().
+type MountOptions struct {
+ // GetFilesystemOptions contains options to FilesystemType.GetFilesystem().
+ GetFilesystemOptions GetFilesystemOptions
+}
+
// OpenOptions contains options to VirtualFilesystem.OpenAt() and
// FilesystemImpl.OpenAt().
type OpenOptions struct {
@@ -114,6 +120,12 @@ type StatOptions struct {
Sync uint32
}
+// UmountOptions contains options to VirtualFilesystem.UmountAt().
+type UmountOptions struct {
+ // Flags contains flags as specified for umount2(2).
+ Flags uint32
+}
+
// WriteOptions contains options to FileDescription.PWrite(),
// FileDescriptionImpl.PWrite(), FileDescription.Write(), and
// FileDescriptionImpl.Write().
diff --git a/pkg/sentry/vfs/pathname.go b/pkg/sentry/vfs/pathname.go
new file mode 100644
index 000000000..8e155654f
--- /dev/null
+++ b/pkg/sentry/vfs/pathname.go
@@ -0,0 +1,153 @@
+// Copyright 2019 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package vfs
+
+import (
+ "sync"
+
+ "gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+var fspathBuilderPool = sync.Pool{
+ New: func() interface{} {
+ return &fspath.Builder{}
+ },
+}
+
+func getFSPathBuilder() *fspath.Builder {
+ return fspathBuilderPool.Get().(*fspath.Builder)
+}
+
+func putFSPathBuilder(b *fspath.Builder) {
+ // No methods can be called on b after b.String(), so reset it to its zero
+ // value (as returned by fspathBuilderPool.New) instead.
+ *b = fspath.Builder{}
+ fspathBuilderPool.Put(b)
+}
+
+// PathnameWithDeleted returns an absolute pathname to vd, consistent with
+// Linux's d_path(). In particular, if vd.Dentry() has been disowned,
+// PathnameWithDeleted appends " (deleted)" to the returned pathname.
+func (vfs *VirtualFilesystem) PathnameWithDeleted(ctx context.Context, vfsroot, vd VirtualDentry) (string, error) {
+ b := getFSPathBuilder()
+ defer putFSPathBuilder(b)
+ haveRef := false
+ defer func() {
+ if haveRef {
+ vd.DecRef()
+ }
+ }()
+
+ origD := vd.dentry
+loop:
+ for {
+ err := vd.mount.fs.impl.PrependPath(ctx, vfsroot, vd, b)
+ switch err.(type) {
+ case nil:
+ if vd.mount == vfsroot.mount && vd.mount.root == vfsroot.dentry {
+ // GenericPrependPath() will have returned
+ // PrependPathAtVFSRootError in this case since it checks
+ // against vfsroot before mnt.root, but other implementations
+ // of FilesystemImpl.PrependPath() may return nil instead.
+ break loop
+ }
+ nextVD := vfs.getMountpointAt(vd.mount, vfsroot)
+ if !nextVD.Ok() {
+ break loop
+ }
+ if haveRef {
+ vd.DecRef()
+ }
+ vd = nextVD
+ haveRef = true
+ // continue loop
+ case PrependPathSyntheticError:
+ // Skip prepending "/" and appending " (deleted)".
+ return b.String(), nil
+ case PrependPathAtVFSRootError, PrependPathAtNonMountRootError:
+ break loop
+ default:
+ return "", err
+ }
+ }
+ b.PrependByte('/')
+ if origD.IsDisowned() {
+ b.AppendString(" (deleted)")
+ }
+ return b.String(), nil
+}
+
+// PathnameForGetcwd returns an absolute pathname to vd, consistent with
+// Linux's sys_getcwd().
+func (vfs *VirtualFilesystem) PathnameForGetcwd(ctx context.Context, vfsroot, vd VirtualDentry) (string, error) {
+ if vd.dentry.IsDisowned() {
+ return "", syserror.ENOENT
+ }
+
+ b := getFSPathBuilder()
+ defer putFSPathBuilder(b)
+ haveRef := false
+ defer func() {
+ if haveRef {
+ vd.DecRef()
+ }
+ }()
+ unreachable := false
+loop:
+ for {
+ err := vd.mount.fs.impl.PrependPath(ctx, vfsroot, vd, b)
+ switch err.(type) {
+ case nil:
+ if vd.mount == vfsroot.mount && vd.mount.root == vfsroot.dentry {
+ break loop
+ }
+ nextVD := vfs.getMountpointAt(vd.mount, vfsroot)
+ if !nextVD.Ok() {
+ unreachable = true
+ break loop
+ }
+ if haveRef {
+ vd.DecRef()
+ }
+ vd = nextVD
+ haveRef = true
+ case PrependPathAtVFSRootError:
+ break loop
+ case PrependPathAtNonMountRootError, PrependPathSyntheticError:
+ unreachable = true
+ break loop
+ default:
+ return "", err
+ }
+ }
+ b.PrependByte('/')
+ if unreachable {
+ b.PrependString("(unreachable)")
+ }
+ return b.String(), nil
+}
+
+// As of this writing, we do not have equivalents to:
+//
+// - d_absolute_path(), which returns EINVAL if (effectively) any call to
+// FilesystemImpl.PrependPath() would return PrependPathAtNonMountRootError.
+//
+// - dentry_path(), which does not walk up mounts (and only returns the path
+// relative to Filesystem root), but also appends "//deleted" for disowned
+// Dentries.
+//
+// These should be added as necessary.
diff --git a/pkg/sentry/vfs/permissions.go b/pkg/sentry/vfs/permissions.go
index f8e74355c..f1edb0680 100644
--- a/pkg/sentry/vfs/permissions.go
+++ b/pkg/sentry/vfs/permissions.go
@@ -119,3 +119,65 @@ func MayWriteFileWithOpenFlags(flags uint32) bool {
return false
}
}
+
+// CheckSetStat checks that creds has permission to change the metadata of a
+// file with the given permissions, UID, and GID as specified by stat, subject
+// to the rules of Linux's fs/attr.c:setattr_prepare().
+func CheckSetStat(creds *auth.Credentials, stat *linux.Statx, mode uint16, kuid auth.KUID, kgid auth.KGID) error {
+ if stat.Mask&linux.STATX_MODE != 0 {
+ if !CanActAsOwner(creds, kuid) {
+ return syserror.EPERM
+ }
+ // TODO(b/30815691): "If the calling process is not privileged (Linux:
+ // does not have the CAP_FSETID capability), and the group of the file
+ // does not match the effective group ID of the process or one of its
+ // supplementary group IDs, the S_ISGID bit will be turned off, but
+ // this will not cause an error to be returned." - chmod(2)
+ }
+ if stat.Mask&linux.STATX_UID != 0 {
+ if !((creds.EffectiveKUID == kuid && auth.KUID(stat.UID) == kuid) ||
+ HasCapabilityOnFile(creds, linux.CAP_CHOWN, kuid, kgid)) {
+ return syserror.EPERM
+ }
+ }
+ if stat.Mask&linux.STATX_GID != 0 {
+ if !((creds.EffectiveKUID == kuid && creds.InGroup(auth.KGID(stat.GID))) ||
+ HasCapabilityOnFile(creds, linux.CAP_CHOWN, kuid, kgid)) {
+ return syserror.EPERM
+ }
+ }
+ if stat.Mask&(linux.STATX_ATIME|linux.STATX_MTIME|linux.STATX_CTIME) != 0 {
+ if !CanActAsOwner(creds, kuid) {
+ if (stat.Mask&linux.STATX_ATIME != 0 && stat.Atime.Nsec != linux.UTIME_NOW) ||
+ (stat.Mask&linux.STATX_MTIME != 0 && stat.Mtime.Nsec != linux.UTIME_NOW) ||
+ (stat.Mask&linux.STATX_CTIME != 0 && stat.Ctime.Nsec != linux.UTIME_NOW) {
+ return syserror.EPERM
+ }
+ // isDir is irrelevant in the following call to
+ // GenericCheckPermissions since ats == MayWrite means that
+ // CAP_DAC_READ_SEARCH does not apply, and CAP_DAC_OVERRIDE
+ // applies, regardless of isDir.
+ if err := GenericCheckPermissions(creds, MayWrite, false /* isDir */, mode, kuid, kgid); err != nil {
+ return err
+ }
+ }
+ }
+ return nil
+}
+
+// CanActAsOwner returns true if creds can act as the owner of a file with the
+// given owning UID, consistent with Linux's
+// fs/inode.c:inode_owner_or_capable().
+func CanActAsOwner(creds *auth.Credentials, kuid auth.KUID) bool {
+ if creds.EffectiveKUID == kuid {
+ return true
+ }
+ return creds.HasCapability(linux.CAP_FOWNER) && creds.UserNamespace.MapFromKUID(kuid).Ok()
+}
+
+// HasCapabilityOnFile returns true if creds has the given capability with
+// respect to a file with the given owning UID and GID, consistent with Linux's
+// kernel/capability.c:capable_wrt_inode_uidgid().
+func HasCapabilityOnFile(creds *auth.Credentials, cp linux.Capability, kuid auth.KUID, kgid auth.KGID) bool {
+ return creds.HasCapability(cp) && creds.UserNamespace.MapFromKUID(kuid).Ok() && creds.UserNamespace.MapFromKGID(kgid).Ok()
+}
diff --git a/pkg/sentry/vfs/syscalls.go b/pkg/sentry/vfs/syscalls.go
deleted file mode 100644
index 436151afa..000000000
--- a/pkg/sentry/vfs/syscalls.go
+++ /dev/null
@@ -1,237 +0,0 @@
-// Copyright 2019 The gVisor Authors.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package vfs
-
-import (
- "gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/sentry/context"
- "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
- "gvisor.dev/gvisor/pkg/syserror"
-)
-
-// PathOperation specifies the path operated on by a VFS method.
-//
-// PathOperation is passed to VFS methods by pointer to reduce memory copying:
-// it's somewhat large and should never escape. (Options structs are passed by
-// pointer to VFS and FileDescription methods for the same reason.)
-type PathOperation struct {
- // Root is the VFS root. References on Root are borrowed from the provider
- // of the PathOperation.
- //
- // Invariants: Root.Ok().
- Root VirtualDentry
-
- // Start is the starting point for the path traversal. References on Start
- // are borrowed from the provider of the PathOperation (i.e. the caller of
- // the VFS method to which the PathOperation was passed).
- //
- // Invariants: Start.Ok(). If Pathname.Absolute, then Start == Root.
- Start VirtualDentry
-
- // Path is the pathname traversed by this operation.
- Pathname string
-
- // If FollowFinalSymlink is true, and the Dentry traversed by the final
- // path component represents a symbolic link, the symbolic link should be
- // followed.
- FollowFinalSymlink bool
-}
-
-// GetDentryAt returns a VirtualDentry representing the given path, at which a
-// file must exist. A reference is taken on the returned VirtualDentry.
-func (vfs *VirtualFilesystem) GetDentryAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *GetDentryOptions) (VirtualDentry, error) {
- rp, err := vfs.getResolvingPath(creds, pop)
- if err != nil {
- return VirtualDentry{}, err
- }
- for {
- d, err := rp.mount.fs.impl.GetDentryAt(ctx, rp, *opts)
- if err == nil {
- vd := VirtualDentry{
- mount: rp.mount,
- dentry: d,
- }
- rp.mount.IncRef()
- vfs.putResolvingPath(rp)
- return vd, nil
- }
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- return VirtualDentry{}, err
- }
- }
-}
-
-// MkdirAt creates a directory at the given path.
-func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *MkdirOptions) error {
- // "Under Linux, apart from the permission bits, the S_ISVTX mode bit is
- // also honored." - mkdir(2)
- opts.Mode &= 01777
- rp, err := vfs.getResolvingPath(creds, pop)
- if err != nil {
- return err
- }
- for {
- err := rp.mount.fs.impl.MkdirAt(ctx, rp, *opts)
- if err == nil {
- vfs.putResolvingPath(rp)
- return nil
- }
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- return err
- }
- }
-}
-
-// MknodAt creates a file of the given mode at the given path. It returns an
-// error from the syserror package.
-func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *MknodOptions) error {
- rp, err := vfs.getResolvingPath(creds, pop)
- if err != nil {
- return nil
- }
- for {
- if err = rp.mount.fs.impl.MknodAt(ctx, rp, *opts); err == nil {
- vfs.putResolvingPath(rp)
- return nil
- }
- // Handle mount traversals.
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- return err
- }
- }
-}
-
-// OpenAt returns a FileDescription providing access to the file at the given
-// path. A reference is taken on the returned FileDescription.
-func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *OpenOptions) (*FileDescription, error) {
- // Remove:
- //
- // - O_LARGEFILE, which we always report in FileDescription status flags
- // since only 64-bit architectures are supported at this time.
- //
- // - O_CLOEXEC, which affects file descriptors and therefore must be
- // handled outside of VFS.
- //
- // - Unknown flags.
- opts.Flags &= linux.O_ACCMODE | linux.O_CREAT | linux.O_EXCL | linux.O_NOCTTY | linux.O_TRUNC | linux.O_APPEND | linux.O_NONBLOCK | linux.O_DSYNC | linux.O_ASYNC | linux.O_DIRECT | linux.O_DIRECTORY | linux.O_NOFOLLOW | linux.O_NOATIME | linux.O_SYNC | linux.O_PATH | linux.O_TMPFILE
- // Linux's __O_SYNC (which we call linux.O_SYNC) implies O_DSYNC.
- if opts.Flags&linux.O_SYNC != 0 {
- opts.Flags |= linux.O_DSYNC
- }
- // Linux's __O_TMPFILE (which we call linux.O_TMPFILE) must be specified
- // with O_DIRECTORY and a writable access mode (to ensure that it fails on
- // filesystem implementations that do not support it).
- if opts.Flags&linux.O_TMPFILE != 0 {
- if opts.Flags&linux.O_DIRECTORY == 0 {
- return nil, syserror.EINVAL
- }
- if opts.Flags&linux.O_CREAT != 0 {
- return nil, syserror.EINVAL
- }
- if opts.Flags&linux.O_ACCMODE == linux.O_RDONLY {
- return nil, syserror.EINVAL
- }
- }
- // O_PATH causes most other flags to be ignored.
- if opts.Flags&linux.O_PATH != 0 {
- opts.Flags &= linux.O_DIRECTORY | linux.O_NOFOLLOW | linux.O_PATH
- }
- // "On Linux, the following bits are also honored in mode: [S_ISUID,
- // S_ISGID, S_ISVTX]" - open(2)
- opts.Mode &= 07777
-
- if opts.Flags&linux.O_NOFOLLOW != 0 {
- pop.FollowFinalSymlink = false
- }
- rp, err := vfs.getResolvingPath(creds, pop)
- if err != nil {
- return nil, err
- }
- if opts.Flags&linux.O_DIRECTORY != 0 {
- rp.mustBeDir = true
- rp.mustBeDirOrig = true
- }
- for {
- fd, err := rp.mount.fs.impl.OpenAt(ctx, rp, *opts)
- if err == nil {
- vfs.putResolvingPath(rp)
- return fd, nil
- }
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- return nil, err
- }
- }
-}
-
-// StatAt returns metadata for the file at the given path.
-func (vfs *VirtualFilesystem) StatAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *StatOptions) (linux.Statx, error) {
- rp, err := vfs.getResolvingPath(creds, pop)
- if err != nil {
- return linux.Statx{}, err
- }
- for {
- stat, err := rp.mount.fs.impl.StatAt(ctx, rp, *opts)
- if err == nil {
- vfs.putResolvingPath(rp)
- return stat, nil
- }
- if !rp.handleError(err) {
- vfs.putResolvingPath(rp)
- return linux.Statx{}, err
- }
- }
-}
-
-// StatusFlags returns file description status flags.
-func (fd *FileDescription) StatusFlags(ctx context.Context) (uint32, error) {
- flags, err := fd.impl.StatusFlags(ctx)
- flags |= linux.O_LARGEFILE
- return flags, err
-}
-
-// SetStatusFlags sets file description status flags.
-func (fd *FileDescription) SetStatusFlags(ctx context.Context, flags uint32) error {
- return fd.impl.SetStatusFlags(ctx, flags)
-}
-
-// TODO:
-//
-// - VFS.SyncAllFilesystems() for sync(2)
-//
-// - Something for syncfs(2)
-//
-// - VFS.LinkAt()
-//
-// - VFS.ReadlinkAt()
-//
-// - VFS.RenameAt()
-//
-// - VFS.RmdirAt()
-//
-// - VFS.SetStatAt()
-//
-// - VFS.StatFSAt()
-//
-// - VFS.SymlinkAt()
-//
-// - VFS.UmountAt()
-//
-// - VFS.UnlinkAt()
-//
-// - FileDescription.(almost everything)
diff --git a/pkg/sentry/vfs/testutil.go b/pkg/sentry/vfs/testutil.go
index 593144cb7..7a1d9e383 100644
--- a/pkg/sentry/vfs/testutil.go
+++ b/pkg/sentry/vfs/testutil.go
@@ -15,7 +15,10 @@
package vfs
import (
+ "fmt"
+
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/context"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/syserror"
@@ -114,6 +117,12 @@ func (fs *FDTestFilesystem) UnlinkAt(ctx context.Context, rp *ResolvingPath) err
return syserror.EPERM
}
+// PrependPath implements FilesystemImpl.PrependPath.
+func (fs *FDTestFilesystem) PrependPath(ctx context.Context, vfsroot, vd VirtualDentry, b *fspath.Builder) error {
+ b.PrependComponent(fmt.Sprintf("vfs.fdTestDentry:%p", vd.dentry.impl.(*fdTestDentry)))
+ return PrependPathSyntheticError{}
+}
+
type fdTestDentry struct {
vfsd Dentry
}
diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go
index f0cd3ffe5..7262b0d0a 100644
--- a/pkg/sentry/vfs/vfs.go
+++ b/pkg/sentry/vfs/vfs.go
@@ -20,6 +20,7 @@
// VirtualFilesystem.mountMu
// Dentry.mu
// Locks acquired by FilesystemImpls between Prepare{Delete,Rename}Dentry and Commit{Delete,Rename*}Dentry
+// VirtualFilesystem.filesystemsMu
// VirtualFilesystem.fsTypesMu
//
// Locking Dentry.mu in multiple Dentries requires holding
@@ -28,6 +29,11 @@ package vfs
import (
"sync"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/syserror"
)
// A VirtualFilesystem (VFS for short) combines Filesystems in trees of Mounts.
@@ -67,6 +73,11 @@ type VirtualFilesystem struct {
// mountpoints is analogous to Linux's mountpoint_hashtable.
mountpoints map[*Dentry]map[*Mount]struct{}
+ // filesystems contains all Filesystems. filesystems is protected by
+ // filesystemsMu.
+ filesystemsMu sync.Mutex
+ filesystems map[*Filesystem]struct{}
+
// fsTypes contains all FilesystemTypes that are usable in the
// VirtualFilesystem. fsTypes is protected by fsTypesMu.
fsTypesMu sync.RWMutex
@@ -77,12 +88,379 @@ type VirtualFilesystem struct {
func New() *VirtualFilesystem {
vfs := &VirtualFilesystem{
mountpoints: make(map[*Dentry]map[*Mount]struct{}),
+ filesystems: make(map[*Filesystem]struct{}),
fsTypes: make(map[string]FilesystemType),
}
vfs.mounts.Init()
return vfs
}
+// PathOperation specifies the path operated on by a VFS method.
+//
+// PathOperation is passed to VFS methods by pointer to reduce memory copying:
+// it's somewhat large and should never escape. (Options structs are passed by
+// pointer to VFS and FileDescription methods for the same reason.)
+type PathOperation struct {
+ // Root is the VFS root. References on Root are borrowed from the provider
+ // of the PathOperation.
+ //
+ // Invariants: Root.Ok().
+ Root VirtualDentry
+
+ // Start is the starting point for the path traversal. References on Start
+ // are borrowed from the provider of the PathOperation (i.e. the caller of
+ // the VFS method to which the PathOperation was passed).
+ //
+ // Invariants: Start.Ok(). If Pathname.Absolute, then Start == Root.
+ Start VirtualDentry
+
+ // Path is the pathname traversed by this operation.
+ Pathname string
+
+ // If FollowFinalSymlink is true, and the Dentry traversed by the final
+ // path component represents a symbolic link, the symbolic link should be
+ // followed.
+ FollowFinalSymlink bool
+}
+
+// GetDentryAt returns a VirtualDentry representing the given path, at which a
+// file must exist. A reference is taken on the returned VirtualDentry.
+func (vfs *VirtualFilesystem) GetDentryAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *GetDentryOptions) (VirtualDentry, error) {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return VirtualDentry{}, err
+ }
+ for {
+ d, err := rp.mount.fs.impl.GetDentryAt(ctx, rp, *opts)
+ if err == nil {
+ vd := VirtualDentry{
+ mount: rp.mount,
+ dentry: d,
+ }
+ rp.mount.IncRef()
+ vfs.putResolvingPath(rp)
+ return vd, nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return VirtualDentry{}, err
+ }
+ }
+}
+
+// LinkAt creates a hard link at newpop representing the existing file at
+// oldpop.
+func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credentials, oldpop, newpop *PathOperation) error {
+ oldVD, err := vfs.GetDentryAt(ctx, creds, oldpop, &GetDentryOptions{})
+ if err != nil {
+ return err
+ }
+ rp, err := vfs.getResolvingPath(creds, newpop)
+ if err != nil {
+ oldVD.DecRef()
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.LinkAt(ctx, rp, oldVD)
+ if err == nil {
+ oldVD.DecRef()
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ oldVD.DecRef()
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// MkdirAt creates a directory at the given path.
+func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *MkdirOptions) error {
+ // "Under Linux, apart from the permission bits, the S_ISVTX mode bit is
+ // also honored." - mkdir(2)
+ opts.Mode &= 0777 | linux.S_ISVTX
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.MkdirAt(ctx, rp, *opts)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// MknodAt creates a file of the given mode at the given path. It returns an
+// error from the syserror package.
+func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *MknodOptions) error {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return nil
+ }
+ for {
+ if err = rp.mount.fs.impl.MknodAt(ctx, rp, *opts); err == nil {
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ // Handle mount traversals.
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// OpenAt returns a FileDescription providing access to the file at the given
+// path. A reference is taken on the returned FileDescription.
+func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *OpenOptions) (*FileDescription, error) {
+ // Remove:
+ //
+ // - O_LARGEFILE, which we always report in FileDescription status flags
+ // since only 64-bit architectures are supported at this time.
+ //
+ // - O_CLOEXEC, which affects file descriptors and therefore must be
+ // handled outside of VFS.
+ //
+ // - Unknown flags.
+ opts.Flags &= linux.O_ACCMODE | linux.O_CREAT | linux.O_EXCL | linux.O_NOCTTY | linux.O_TRUNC | linux.O_APPEND | linux.O_NONBLOCK | linux.O_DSYNC | linux.O_ASYNC | linux.O_DIRECT | linux.O_DIRECTORY | linux.O_NOFOLLOW | linux.O_NOATIME | linux.O_SYNC | linux.O_PATH | linux.O_TMPFILE
+ // Linux's __O_SYNC (which we call linux.O_SYNC) implies O_DSYNC.
+ if opts.Flags&linux.O_SYNC != 0 {
+ opts.Flags |= linux.O_DSYNC
+ }
+ // Linux's __O_TMPFILE (which we call linux.O_TMPFILE) must be specified
+ // with O_DIRECTORY and a writable access mode (to ensure that it fails on
+ // filesystem implementations that do not support it).
+ if opts.Flags&linux.O_TMPFILE != 0 {
+ if opts.Flags&linux.O_DIRECTORY == 0 {
+ return nil, syserror.EINVAL
+ }
+ if opts.Flags&linux.O_CREAT != 0 {
+ return nil, syserror.EINVAL
+ }
+ if opts.Flags&linux.O_ACCMODE == linux.O_RDONLY {
+ return nil, syserror.EINVAL
+ }
+ }
+ // O_PATH causes most other flags to be ignored.
+ if opts.Flags&linux.O_PATH != 0 {
+ opts.Flags &= linux.O_DIRECTORY | linux.O_NOFOLLOW | linux.O_PATH
+ }
+ // "On Linux, the following bits are also honored in mode: [S_ISUID,
+ // S_ISGID, S_ISVTX]" - open(2)
+ opts.Mode &= 0777 | linux.S_ISUID | linux.S_ISGID | linux.S_ISVTX
+
+ if opts.Flags&linux.O_NOFOLLOW != 0 {
+ pop.FollowFinalSymlink = false
+ }
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return nil, err
+ }
+ if opts.Flags&linux.O_DIRECTORY != 0 {
+ rp.mustBeDir = true
+ rp.mustBeDirOrig = true
+ }
+ for {
+ fd, err := rp.mount.fs.impl.OpenAt(ctx, rp, *opts)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return fd, nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return nil, err
+ }
+ }
+}
+
+// ReadlinkAt returns the target of the symbolic link at the given path.
+func (vfs *VirtualFilesystem) ReadlinkAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation) (string, error) {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return "", err
+ }
+ for {
+ target, err := rp.mount.fs.impl.ReadlinkAt(ctx, rp)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return target, nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return "", err
+ }
+ }
+}
+
+// RenameAt renames the file at oldpop to newpop.
+func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credentials, oldpop, newpop *PathOperation, opts *RenameOptions) error {
+ oldVD, err := vfs.GetDentryAt(ctx, creds, oldpop, &GetDentryOptions{})
+ if err != nil {
+ return err
+ }
+ rp, err := vfs.getResolvingPath(creds, newpop)
+ if err != nil {
+ oldVD.DecRef()
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.RenameAt(ctx, rp, oldVD, *opts)
+ if err == nil {
+ oldVD.DecRef()
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ oldVD.DecRef()
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// RmdirAt removes the directory at the given path.
+func (vfs *VirtualFilesystem) RmdirAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation) error {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.RmdirAt(ctx, rp)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// SetStatAt changes metadata for the file at the given path.
+func (vfs *VirtualFilesystem) SetStatAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *SetStatOptions) error {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.SetStatAt(ctx, rp, *opts)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// StatAt returns metadata for the file at the given path.
+func (vfs *VirtualFilesystem) StatAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *StatOptions) (linux.Statx, error) {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ for {
+ stat, err := rp.mount.fs.impl.StatAt(ctx, rp, *opts)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return stat, nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return linux.Statx{}, err
+ }
+ }
+}
+
+// StatFSAt returns metadata for the filesystem containing the file at the
+// given path.
+func (vfs *VirtualFilesystem) StatFSAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation) (linux.Statfs, error) {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return linux.Statfs{}, err
+ }
+ for {
+ statfs, err := rp.mount.fs.impl.StatFSAt(ctx, rp)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return statfs, nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return linux.Statfs{}, err
+ }
+ }
+}
+
+// SymlinkAt creates a symbolic link at the given path with the given target.
+func (vfs *VirtualFilesystem) SymlinkAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, target string) error {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.SymlinkAt(ctx, rp, target)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// UnlinkAt deletes the non-directory file at the given path.
+func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation) error {
+ rp, err := vfs.getResolvingPath(creds, pop)
+ if err != nil {
+ return err
+ }
+ for {
+ err := rp.mount.fs.impl.UnlinkAt(ctx, rp)
+ if err == nil {
+ vfs.putResolvingPath(rp)
+ return nil
+ }
+ if !rp.handleError(err) {
+ vfs.putResolvingPath(rp)
+ return err
+ }
+ }
+}
+
+// SyncAllFilesystems has the semantics of Linux's sync(2).
+func (vfs *VirtualFilesystem) SyncAllFilesystems(ctx context.Context) error {
+ fss := make(map[*Filesystem]struct{})
+ vfs.filesystemsMu.Lock()
+ for fs := range vfs.filesystems {
+ if !fs.TryIncRef() {
+ continue
+ }
+ fss[fs] = struct{}{}
+ }
+ vfs.filesystemsMu.Unlock()
+ var retErr error
+ for fs := range fss {
+ if err := fs.impl.Sync(ctx); err != nil && retErr == nil {
+ retErr = err
+ }
+ fs.DecRef()
+ }
+ return retErr
+}
+
// A VirtualDentry represents a node in a VFS tree, by combining a Dentry
// (which represents a node in a Filesystem's tree) and a Mount (which
// represents the Filesystem's position in a VFS mount tree).