summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go12
-rw-r--r--pkg/sentry/fsimpl/kernfs/filesystem.go10
-rw-r--r--pkg/sentry/fsimpl/overlay/BUILD41
-rw-r--r--pkg/sentry/fsimpl/overlay/copy_up.go262
-rw-r--r--pkg/sentry/fsimpl/overlay/directory.go265
-rw-r--r--pkg/sentry/fsimpl/overlay/filesystem.go1364
-rw-r--r--pkg/sentry/fsimpl/overlay/non_directory.go266
-rw-r--r--pkg/sentry/fsimpl/overlay/overlay.go612
-rw-r--r--pkg/sentry/vfs/file_description.go17
-rw-r--r--pkg/syserror/syserror.go1
10 files changed, 2837 insertions, 13 deletions
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 40933b74b..3c467e313 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -118,7 +118,7 @@ func putDentrySlice(ds *[]*dentry) {
// must be up to date.
//
// Postconditions: The returned dentry's cached metadata is up to date.
-func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, ds **[]*dentry) (*dentry, error) {
+func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, mayFollowSymlinks bool, ds **[]*dentry) (*dentry, error) {
if !d.isDir() {
return nil, syserror.ENOTDIR
}
@@ -168,7 +168,7 @@ afterSymlink:
if err := rp.CheckMount(&child.vfsd); err != nil {
return nil, err
}
- if child.isSymlink() && rp.ShouldFollowSymlink() {
+ if child.isSymlink() && mayFollowSymlinks && rp.ShouldFollowSymlink() {
target, err := child.readlink(ctx, rp.Mount())
if err != nil {
return nil, err
@@ -275,7 +275,7 @@ func (fs *filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.Vir
func (fs *filesystem) walkParentDirLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, ds **[]*dentry) (*dentry, error) {
for !rp.Final() {
d.dirMu.Lock()
- next, err := fs.stepLocked(ctx, rp, d, ds)
+ next, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
d.dirMu.Unlock()
if err != nil {
return nil, err
@@ -301,7 +301,7 @@ func (fs *filesystem) resolveLocked(ctx context.Context, rp *vfs.ResolvingPath,
}
for !rp.Done() {
d.dirMu.Lock()
- next, err := fs.stepLocked(ctx, rp, d, ds)
+ next, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
d.dirMu.Unlock()
if err != nil {
return nil, err
@@ -754,7 +754,7 @@ afterTrailingSymlink:
}
// Determine whether or not we need to create a file.
parent.dirMu.Lock()
- child, err := fs.stepLocked(ctx, rp, parent, &ds)
+ child, err := fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
if err == syserror.ENOENT && mayCreate {
if parent.isSynthetic() {
parent.dirMu.Unlock()
@@ -939,7 +939,7 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
// Filter file creation flags and O_LARGEFILE out; the create RPC already
// has the semantics of O_CREAT|O_EXCL, while some servers will choke on
// O_LARGEFILE.
- createFlags := p9.OpenFlags(opts.Flags &^ (linux.O_CREAT | linux.O_EXCL | linux.O_NOCTTY | linux.O_TRUNC | linux.O_LARGEFILE))
+ createFlags := p9.OpenFlags(opts.Flags &^ (vfs.FileCreationFlags | linux.O_LARGEFILE))
fdobj, openFile, createQID, _, err := dirfile.create(ctx, name, createFlags, (p9.FileMode)(opts.Mode), (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
if err != nil {
dirfile.close(ctx)
diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go
index 4a12ae245..8939871c1 100644
--- a/pkg/sentry/fsimpl/kernfs/filesystem.go
+++ b/pkg/sentry/fsimpl/kernfs/filesystem.go
@@ -35,7 +35,7 @@ import (
// Preconditions: Filesystem.mu must be locked for at least reading. !rp.Done().
//
// Postcondition: Caller must call fs.processDeferredDecRefs*.
-func (fs *Filesystem) stepExistingLocked(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry) (*vfs.Dentry, error) {
+func (fs *Filesystem) stepExistingLocked(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, mayFollowSymlinks bool) (*vfs.Dentry, error) {
d := vfsd.Impl().(*Dentry)
if !d.isDir() {
return nil, syserror.ENOTDIR
@@ -81,7 +81,7 @@ afterSymlink:
return nil, err
}
// Resolve any symlink at current path component.
- if rp.ShouldFollowSymlink() && next.isSymlink() {
+ if mayFollowSymlinks && rp.ShouldFollowSymlink() && next.isSymlink() {
targetVD, targetPathname, err := next.inode.Getlink(ctx, rp.Mount())
if err != nil {
return nil, err
@@ -152,7 +152,7 @@ func (fs *Filesystem) walkExistingLocked(ctx context.Context, rp *vfs.ResolvingP
vfsd := rp.Start()
for !rp.Done() {
var err error
- vfsd, err = fs.stepExistingLocked(ctx, rp, vfsd)
+ vfsd, err = fs.stepExistingLocked(ctx, rp, vfsd, true /* mayFollowSymlinks */)
if err != nil {
return nil, nil, err
}
@@ -178,7 +178,7 @@ func (fs *Filesystem) walkParentDirLocked(ctx context.Context, rp *vfs.Resolving
vfsd := rp.Start()
for !rp.Final() {
var err error
- vfsd, err = fs.stepExistingLocked(ctx, rp, vfsd)
+ vfsd, err = fs.stepExistingLocked(ctx, rp, vfsd, true /* mayFollowSymlinks */)
if err != nil {
return nil, nil, err
}
@@ -449,7 +449,7 @@ afterTrailingSymlink:
return nil, syserror.ENAMETOOLONG
}
// Determine whether or not we need to create a file.
- childVFSD, err := fs.stepExistingLocked(ctx, rp, parentVFSD)
+ childVFSD, err := fs.stepExistingLocked(ctx, rp, parentVFSD, false /* mayFollowSymlinks */)
if err == syserror.ENOENT {
// Already checked for searchability above; now check for writability.
if err := parentInode.CheckPermissions(ctx, rp.Credentials(), vfs.MayWrite); err != nil {
diff --git a/pkg/sentry/fsimpl/overlay/BUILD b/pkg/sentry/fsimpl/overlay/BUILD
new file mode 100644
index 000000000..f9413bbdd
--- /dev/null
+++ b/pkg/sentry/fsimpl/overlay/BUILD
@@ -0,0 +1,41 @@
+load("//tools:defs.bzl", "go_library")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
+
+licenses(["notice"])
+
+go_template_instance(
+ name = "fstree",
+ out = "fstree.go",
+ package = "overlay",
+ prefix = "generic",
+ template = "//pkg/sentry/vfs/genericfstree:generic_fstree",
+ types = {
+ "Dentry": "dentry",
+ },
+)
+
+go_library(
+ name = "overlay",
+ srcs = [
+ "copy_up.go",
+ "directory.go",
+ "filesystem.go",
+ "fstree.go",
+ "non_directory.go",
+ "overlay.go",
+ ],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/fspath",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/memmap",
+ "//pkg/sentry/socket/unix/transport",
+ "//pkg/sentry/vfs",
+ "//pkg/sentry/vfs/lock",
+ "//pkg/sync",
+ "//pkg/syserror",
+ "//pkg/usermem",
+ ],
+)
diff --git a/pkg/sentry/fsimpl/overlay/copy_up.go b/pkg/sentry/fsimpl/overlay/copy_up.go
new file mode 100644
index 000000000..8f8dcfafe
--- /dev/null
+++ b/pkg/sentry/fsimpl/overlay/copy_up.go
@@ -0,0 +1,262 @@
+// Copyright 2020 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 overlay
+
+import (
+ "fmt"
+ "io"
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+func (d *dentry) isCopiedUp() bool {
+ return atomic.LoadUint32(&d.copiedUp) != 0
+}
+
+// copyUpLocked ensures that d exists on the upper layer, i.e. d.upperVD.Ok().
+//
+// Preconditions: filesystem.renameMu must be locked.
+func (d *dentry) copyUpLocked(ctx context.Context) error {
+ // Fast path.
+ if d.isCopiedUp() {
+ return nil
+ }
+
+ ftype := atomic.LoadUint32(&d.mode) & linux.S_IFMT
+ switch ftype {
+ case linux.S_IFREG, linux.S_IFDIR, linux.S_IFLNK, linux.S_IFBLK, linux.S_IFCHR:
+ // Can be copied-up.
+ default:
+ // Can't be copied-up.
+ return syserror.EPERM
+ }
+
+ // Ensure that our parent directory is copied-up.
+ if d.parent == nil {
+ // d is a filesystem root with no upper layer.
+ return syserror.EROFS
+ }
+ if err := d.parent.copyUpLocked(ctx); err != nil {
+ return err
+ }
+
+ d.copyMu.Lock()
+ defer d.copyMu.Unlock()
+ if d.upperVD.Ok() {
+ // Raced with another call to d.copyUpLocked().
+ return nil
+ }
+ if d.vfsd.IsDead() {
+ // Raced with deletion of d.
+ return syserror.ENOENT
+ }
+
+ // Perform copy-up.
+ vfsObj := d.fs.vfsfs.VirtualFilesystem()
+ newpop := vfs.PathOperation{
+ Root: d.parent.upperVD,
+ Start: d.parent.upperVD,
+ Path: fspath.Parse(d.name),
+ }
+ cleanupUndoCopyUp := func() {
+ var err error
+ if ftype == linux.S_IFDIR {
+ err = vfsObj.RmdirAt(ctx, d.fs.creds, &newpop)
+ } else {
+ err = vfsObj.UnlinkAt(ctx, d.fs.creds, &newpop)
+ }
+ if err != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer file after copy-up error: %v", err)
+ }
+ }
+ switch ftype {
+ case linux.S_IFREG:
+ oldFD, err := vfsObj.OpenAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: d.lowerVDs[0],
+ Start: d.lowerVDs[0],
+ }, &vfs.OpenOptions{
+ Flags: linux.O_RDONLY,
+ })
+ if err != nil {
+ return err
+ }
+ defer oldFD.DecRef()
+ newFD, err := vfsObj.OpenAt(ctx, d.fs.creds, &newpop, &vfs.OpenOptions{
+ Flags: linux.O_WRONLY | linux.O_CREAT | linux.O_EXCL,
+ Mode: linux.FileMode(d.mode &^ linux.S_IFMT),
+ })
+ if err != nil {
+ return err
+ }
+ defer newFD.DecRef()
+ bufIOSeq := usermem.BytesIOSequence(make([]byte, 32*1024)) // arbitrary buffer size
+ for {
+ readN, readErr := oldFD.Read(ctx, bufIOSeq, vfs.ReadOptions{})
+ if readErr != nil && readErr != io.EOF {
+ cleanupUndoCopyUp()
+ return readErr
+ }
+ total := int64(0)
+ for total < readN {
+ writeN, writeErr := newFD.Write(ctx, bufIOSeq.DropFirst64(total), vfs.WriteOptions{})
+ total += writeN
+ if writeErr != nil {
+ cleanupUndoCopyUp()
+ return writeErr
+ }
+ }
+ if readErr == io.EOF {
+ break
+ }
+ }
+ if err := newFD.SetStat(ctx, vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: d.uid,
+ GID: d.gid,
+ },
+ }); err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ d.upperVD = newFD.VirtualDentry()
+ d.upperVD.IncRef()
+
+ case linux.S_IFDIR:
+ if err := vfsObj.MkdirAt(ctx, d.fs.creds, &newpop, &vfs.MkdirOptions{
+ Mode: linux.FileMode(d.mode &^ linux.S_IFMT),
+ }); err != nil {
+ return err
+ }
+ if err := vfsObj.SetStatAt(ctx, d.fs.creds, &newpop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: d.uid,
+ GID: d.gid,
+ },
+ }); err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ upperVD, err := vfsObj.GetDentryAt(ctx, d.fs.creds, &newpop, &vfs.GetDentryOptions{})
+ if err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ d.upperVD = upperVD
+
+ case linux.S_IFLNK:
+ target, err := vfsObj.ReadlinkAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: d.lowerVDs[0],
+ Start: d.lowerVDs[0],
+ })
+ if err != nil {
+ return err
+ }
+ if err := vfsObj.SymlinkAt(ctx, d.fs.creds, &newpop, target); err != nil {
+ return err
+ }
+ if err := vfsObj.SetStatAt(ctx, d.fs.creds, &newpop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID,
+ Mode: uint16(d.mode),
+ UID: d.uid,
+ GID: d.gid,
+ },
+ }); err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ upperVD, err := vfsObj.GetDentryAt(ctx, d.fs.creds, &newpop, &vfs.GetDentryOptions{})
+ if err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ d.upperVD = upperVD
+
+ case linux.S_IFBLK, linux.S_IFCHR:
+ lowerStat, err := vfsObj.StatAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: d.lowerVDs[0],
+ Start: d.lowerVDs[0],
+ }, &vfs.StatOptions{})
+ if err != nil {
+ return err
+ }
+ if err := vfsObj.MknodAt(ctx, d.fs.creds, &newpop, &vfs.MknodOptions{
+ Mode: linux.FileMode(d.mode),
+ DevMajor: lowerStat.RdevMajor,
+ DevMinor: lowerStat.RdevMinor,
+ }); err != nil {
+ return err
+ }
+ if err := vfsObj.SetStatAt(ctx, d.fs.creds, &newpop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: d.uid,
+ GID: d.gid,
+ },
+ }); err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ upperVD, err := vfsObj.GetDentryAt(ctx, d.fs.creds, &newpop, &vfs.GetDentryOptions{})
+ if err != nil {
+ cleanupUndoCopyUp()
+ return err
+ }
+ d.upperVD = upperVD
+
+ default:
+ // Should have rejected this at the beginning of this function?
+ panic(fmt.Sprintf("unexpected file type %o", ftype))
+ }
+
+ // TODO(gvisor.dev/issue/1199): copy up xattrs
+
+ // Update the dentry's device and inode numbers (except for directories,
+ // for which these remain overlay-assigned).
+ if ftype != linux.S_IFDIR {
+ upperStat, err := vfsObj.StatAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: d.upperVD,
+ Start: d.upperVD,
+ }, &vfs.StatOptions{
+ Mask: linux.STATX_INO,
+ })
+ if err != nil {
+ d.upperVD.DecRef()
+ d.upperVD = vfs.VirtualDentry{}
+ cleanupUndoCopyUp()
+ return err
+ }
+ if upperStat.Mask&linux.STATX_INO == 0 {
+ d.upperVD.DecRef()
+ d.upperVD = vfs.VirtualDentry{}
+ cleanupUndoCopyUp()
+ return syserror.EREMOTE
+ }
+ atomic.StoreUint32(&d.devMajor, upperStat.DevMajor)
+ atomic.StoreUint32(&d.devMinor, upperStat.DevMinor)
+ atomic.StoreUint64(&d.ino, upperStat.Ino)
+ }
+
+ atomic.StoreUint32(&d.copiedUp, 1)
+ return nil
+}
diff --git a/pkg/sentry/fsimpl/overlay/directory.go b/pkg/sentry/fsimpl/overlay/directory.go
new file mode 100644
index 000000000..6f47167d3
--- /dev/null
+++ b/pkg/sentry/fsimpl/overlay/directory.go
@@ -0,0 +1,265 @@
+// Copyright 2020 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 overlay
+
+import (
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+func (d *dentry) isDir() bool {
+ return atomic.LoadUint32(&d.mode)&linux.S_IFMT == linux.S_IFDIR
+}
+
+// Preconditions: d.dirMu must be locked. d.isDir().
+func (d *dentry) collectWhiteoutsForRmdirLocked(ctx context.Context) (map[string]bool, error) {
+ vfsObj := d.fs.vfsfs.VirtualFilesystem()
+ var readdirErr error
+ whiteouts := make(map[string]bool)
+ var maybeWhiteouts []string
+ d.iterLayers(func(layerVD vfs.VirtualDentry, isUpper bool) bool {
+ layerFD, err := vfsObj.OpenAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ }, &vfs.OpenOptions{
+ Flags: linux.O_RDONLY | linux.O_DIRECTORY,
+ })
+ if err != nil {
+ readdirErr = err
+ return false
+ }
+ defer layerFD.DecRef()
+
+ // Reuse slice allocated for maybeWhiteouts from a previous layer to
+ // reduce allocations.
+ maybeWhiteouts = maybeWhiteouts[:0]
+ if err := layerFD.IterDirents(ctx, vfs.IterDirentsCallbackFunc(func(dirent vfs.Dirent) error {
+ if dirent.Name == "." || dirent.Name == ".." {
+ return nil
+ }
+ if _, ok := whiteouts[dirent.Name]; ok {
+ // This file has been whited-out in a previous layer.
+ return nil
+ }
+ if dirent.Type == linux.DT_CHR {
+ // We have to determine if this is a whiteout, which doesn't
+ // count against the directory's emptiness. However, we can't
+ // do so while holding locks held by layerFD.IterDirents().
+ maybeWhiteouts = append(maybeWhiteouts, dirent.Name)
+ return nil
+ }
+ // Non-whiteout file in the directory prevents rmdir.
+ return syserror.ENOTEMPTY
+ })); err != nil {
+ readdirErr = err
+ return false
+ }
+
+ for _, maybeWhiteoutName := range maybeWhiteouts {
+ stat, err := vfsObj.StatAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ Path: fspath.Parse(maybeWhiteoutName),
+ }, &vfs.StatOptions{})
+ if err != nil {
+ readdirErr = err
+ return false
+ }
+ if stat.RdevMajor != 0 || stat.RdevMinor != 0 {
+ // This file is a real character device, not a whiteout.
+ readdirErr = syserror.ENOTEMPTY
+ return false
+ }
+ whiteouts[maybeWhiteoutName] = isUpper
+ }
+ // Continue iteration since we haven't found any non-whiteout files in
+ // this directory yet.
+ return true
+ })
+ return whiteouts, readdirErr
+}
+
+type directoryFD struct {
+ fileDescription
+ vfs.DirectoryFileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ mu sync.Mutex
+ off int64
+ dirents []vfs.Dirent
+}
+
+// Release implements vfs.FileDescriptionImpl.Release.
+func (fd *directoryFD) Release() {
+}
+
+// IterDirents implements vfs.FileDescriptionImpl.IterDirents.
+func (fd *directoryFD) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback) error {
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
+
+ d := fd.dentry()
+ if fd.dirents == nil {
+ ds, err := d.getDirents(ctx)
+ if err != nil {
+ return err
+ }
+ fd.dirents = ds
+ }
+
+ for fd.off < int64(len(fd.dirents)) {
+ if err := cb.Handle(fd.dirents[fd.off]); err != nil {
+ return err
+ }
+ fd.off++
+ }
+ return nil
+}
+
+// Preconditions: d.isDir().
+func (d *dentry) getDirents(ctx context.Context) ([]vfs.Dirent, error) {
+ d.fs.renameMu.RLock()
+ defer d.fs.renameMu.RUnlock()
+ d.dirMu.Lock()
+ defer d.dirMu.Unlock()
+
+ if d.dirents != nil {
+ return d.dirents, nil
+ }
+
+ parent := genericParentOrSelf(d)
+ dirents := []vfs.Dirent{
+ {
+ Name: ".",
+ Type: linux.DT_DIR,
+ Ino: d.ino,
+ NextOff: 1,
+ },
+ {
+ Name: "..",
+ Type: uint8(atomic.LoadUint32(&parent.mode) >> 12),
+ Ino: parent.ino,
+ NextOff: 2,
+ },
+ }
+
+ // Merge dirents from all layers comprising this directory.
+ vfsObj := d.fs.vfsfs.VirtualFilesystem()
+ var readdirErr error
+ prevDirents := make(map[string]struct{})
+ var maybeWhiteouts []vfs.Dirent
+ d.iterLayers(func(layerVD vfs.VirtualDentry, isUpper bool) bool {
+ layerFD, err := vfsObj.OpenAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ }, &vfs.OpenOptions{
+ Flags: linux.O_RDONLY | linux.O_DIRECTORY,
+ })
+ if err != nil {
+ readdirErr = err
+ return false
+ }
+ defer layerFD.DecRef()
+
+ // Reuse slice allocated for maybeWhiteouts from a previous layer to
+ // reduce allocations.
+ maybeWhiteouts = maybeWhiteouts[:0]
+ if err := layerFD.IterDirents(ctx, vfs.IterDirentsCallbackFunc(func(dirent vfs.Dirent) error {
+ if dirent.Name == "." || dirent.Name == ".." {
+ return nil
+ }
+ if _, ok := prevDirents[dirent.Name]; ok {
+ // This file is hidden by, or merged with, another file with
+ // the same name in a previous layer.
+ return nil
+ }
+ prevDirents[dirent.Name] = struct{}{}
+ if dirent.Type == linux.DT_CHR {
+ // We can't determine if this file is a whiteout while holding
+ // locks held by layerFD.IterDirents().
+ maybeWhiteouts = append(maybeWhiteouts, dirent)
+ return nil
+ }
+ dirent.NextOff = int64(len(dirents) + 1)
+ dirents = append(dirents, dirent)
+ return nil
+ })); err != nil {
+ readdirErr = err
+ return false
+ }
+
+ for _, dirent := range maybeWhiteouts {
+ stat, err := vfsObj.StatAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ Path: fspath.Parse(dirent.Name),
+ }, &vfs.StatOptions{})
+ if err != nil {
+ readdirErr = err
+ return false
+ }
+ if stat.RdevMajor == 0 && stat.RdevMinor == 0 {
+ // This file is a whiteout; don't emit a dirent for it.
+ continue
+ }
+ dirent.NextOff = int64(len(dirents) + 1)
+ dirents = append(dirents, dirent)
+ }
+ return true
+ })
+ if readdirErr != nil {
+ return nil, readdirErr
+ }
+
+ // Cache dirents for future directoryFDs.
+ d.dirents = dirents
+ return dirents, nil
+}
+
+// Seek implements vfs.FileDescriptionImpl.Seek.
+func (fd *directoryFD) Seek(ctx context.Context, offset int64, whence int32) (int64, error) {
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
+
+ switch whence {
+ case linux.SEEK_SET:
+ if offset < 0 {
+ return 0, syserror.EINVAL
+ }
+ if offset == 0 {
+ // Ensure that the next call to fd.IterDirents() calls
+ // fd.dentry().getDirents().
+ fd.dirents = nil
+ }
+ fd.off = offset
+ return fd.off, nil
+ case linux.SEEK_CUR:
+ offset += fd.off
+ if offset < 0 {
+ return 0, syserror.EINVAL
+ }
+ // Don't clear fd.dirents in this case, even if offset == 0.
+ fd.off = offset
+ return fd.off, nil
+ default:
+ return 0, syserror.EINVAL
+ }
+}
diff --git a/pkg/sentry/fsimpl/overlay/filesystem.go b/pkg/sentry/fsimpl/overlay/filesystem.go
new file mode 100644
index 000000000..ff82e1f20
--- /dev/null
+++ b/pkg/sentry/fsimpl/overlay/filesystem.go
@@ -0,0 +1,1364 @@
+// Copyright 2020 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 overlay
+
+import (
+ "sync/atomic"
+
+ "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/sentry/socket/unix/transport"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+// _OVL_XATTR_OPAQUE is an extended attribute key whose value is set to "y" for
+// opaque directories.
+// Linux: fs/overlayfs/overlayfs.h:OVL_XATTR_OPAQUE
+const _OVL_XATTR_OPAQUE = "trusted.overlay.opaque"
+
+func isWhiteout(stat *linux.Statx) bool {
+ return stat.Mode&linux.S_IFMT == linux.S_IFCHR && stat.RdevMajor == 0 && stat.RdevMinor == 0
+}
+
+// Sync implements vfs.FilesystemImpl.Sync.
+func (fs *filesystem) Sync(ctx context.Context) error {
+ if fs.opts.UpperRoot.Ok() {
+ return fs.opts.UpperRoot.Mount().Filesystem().Impl().Sync(ctx)
+ }
+ return nil
+}
+
+var dentrySlicePool = sync.Pool{
+ New: func() interface{} {
+ ds := make([]*dentry, 0, 4) // arbitrary non-zero initial capacity
+ return &ds
+ },
+}
+
+func appendDentry(ds *[]*dentry, d *dentry) *[]*dentry {
+ if ds == nil {
+ ds = dentrySlicePool.Get().(*[]*dentry)
+ }
+ *ds = append(*ds, d)
+ return ds
+}
+
+// Preconditions: ds != nil.
+func putDentrySlice(ds *[]*dentry) {
+ // Allow dentries to be GC'd.
+ for i := range *ds {
+ (*ds)[i] = nil
+ }
+ *ds = (*ds)[:0]
+ dentrySlicePool.Put(ds)
+}
+
+// renameMuRUnlockAndCheckDrop calls fs.renameMu.RUnlock(), then calls
+// dentry.checkDropLocked on all dentries in *ds with fs.renameMu locked for
+// writing.
+//
+// ds is a pointer-to-pointer since defer evaluates its arguments immediately,
+// but dentry slices are allocated lazily, and it's much easier to say "defer
+// fs.renameMuRUnlockAndCheckDrop(&ds)" than "defer func() {
+// fs.renameMuRUnlockAndCheckDrop(ds) }()" to work around this.
+func (fs *filesystem) renameMuRUnlockAndCheckDrop(ds **[]*dentry) {
+ fs.renameMu.RUnlock()
+ if *ds == nil {
+ return
+ }
+ if len(**ds) != 0 {
+ fs.renameMu.Lock()
+ for _, d := range **ds {
+ d.checkDropLocked()
+ }
+ fs.renameMu.Unlock()
+ }
+ putDentrySlice(*ds)
+}
+
+func (fs *filesystem) renameMuUnlockAndCheckDrop(ds **[]*dentry) {
+ if *ds == nil {
+ fs.renameMu.Unlock()
+ return
+ }
+ for _, d := range **ds {
+ d.checkDropLocked()
+ }
+ fs.renameMu.Unlock()
+ putDentrySlice(*ds)
+}
+
+// stepLocked resolves rp.Component() to an existing file, starting from the
+// given directory.
+//
+// Dentries which may have a reference count of zero, and which therefore
+// should be dropped once traversal is complete, are appended to ds.
+//
+// Preconditions: fs.renameMu must be locked. d.dirMu must be locked.
+// !rp.Done().
+func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, mayFollowSymlinks bool, ds **[]*dentry) (*dentry, error) {
+ if !d.isDir() {
+ return nil, syserror.ENOTDIR
+ }
+ if err := d.checkPermissions(rp.Credentials(), vfs.MayExec); err != nil {
+ return nil, err
+ }
+afterSymlink:
+ name := rp.Component()
+ if name == "." {
+ rp.Advance()
+ return d, nil
+ }
+ if name == ".." {
+ if isRoot, err := rp.CheckRoot(&d.vfsd); err != nil {
+ return nil, err
+ } else if isRoot || d.parent == nil {
+ rp.Advance()
+ return d, nil
+ }
+ if err := rp.CheckMount(&d.parent.vfsd); err != nil {
+ return nil, err
+ }
+ rp.Advance()
+ return d.parent, nil
+ }
+ child, err := fs.getChildLocked(ctx, d, name, ds)
+ if err != nil {
+ return nil, err
+ }
+ if err := rp.CheckMount(&child.vfsd); err != nil {
+ return nil, err
+ }
+ if child.isSymlink() && mayFollowSymlinks && rp.ShouldFollowSymlink() {
+ target, err := child.readlink(ctx)
+ if err != nil {
+ return nil, err
+ }
+ if err := rp.HandleSymlink(target); err != nil {
+ return nil, err
+ }
+ goto afterSymlink // don't check the current directory again
+ }
+ rp.Advance()
+ return child, nil
+}
+
+// Preconditions: fs.renameMu must be locked. d.dirMu must be locked.
+func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name string, ds **[]*dentry) (*dentry, error) {
+ if child, ok := parent.children[name]; ok {
+ return child, nil
+ }
+ child, err := fs.lookupLocked(ctx, parent, name)
+ if err != nil {
+ return nil, err
+ }
+ if parent.children == nil {
+ parent.children = make(map[string]*dentry)
+ }
+ parent.children[name] = child
+ // child's refcount is initially 0, so it may be dropped after traversal.
+ *ds = appendDentry(*ds, child)
+ return child, nil
+}
+
+// Preconditions: fs.renameMu must be locked. parent.dirMu must be locked.
+func (fs *filesystem) lookupLocked(ctx context.Context, parent *dentry, name string) (*dentry, error) {
+ childPath := fspath.Parse(name)
+ child := fs.newDentry()
+ existsOnAnyLayer := false
+ var lookupErr error
+
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ parent.iterLayers(func(parentVD vfs.VirtualDentry, isUpper bool) bool {
+ childVD, err := vfsObj.GetDentryAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: parentVD,
+ Start: parentVD,
+ Path: childPath,
+ }, &vfs.GetDentryOptions{})
+ if err == syserror.ENOENT || err == syserror.ENAMETOOLONG {
+ // The file doesn't exist on this layer. Proceed to the next one.
+ return true
+ }
+ if err != nil {
+ lookupErr = err
+ return false
+ }
+
+ mask := uint32(linux.STATX_TYPE)
+ if !existsOnAnyLayer {
+ // Mode, UID, GID, and (for non-directories) inode number come from
+ // the topmost layer on which the file exists.
+ mask |= linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID | linux.STATX_INO
+ }
+ stat, err := vfsObj.StatAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: childVD,
+ Start: childVD,
+ }, &vfs.StatOptions{
+ Mask: mask,
+ })
+ if err != nil {
+ lookupErr = err
+ return false
+ }
+ if stat.Mask&mask != mask {
+ lookupErr = syserror.EREMOTE
+ return false
+ }
+
+ if isWhiteout(&stat) {
+ // This is a whiteout, so it "doesn't exist" on this layer, and
+ // layers below this one are ignored.
+ return false
+ }
+ isDir := stat.Mode&linux.S_IFMT == linux.S_IFDIR
+ if existsOnAnyLayer && !isDir {
+ // Directories are not merged with non-directory files from lower
+ // layers; instead, layers including and below the first
+ // non-directory file are ignored. (This file must be a directory
+ // on previous layers, since lower layers aren't searched for
+ // non-directory files.)
+ return false
+ }
+
+ // Update child to include this layer.
+ if isUpper {
+ child.upperVD = childVD
+ child.copiedUp = 1
+ } else {
+ child.lowerVDs = append(child.lowerVDs, childVD)
+ }
+ if !existsOnAnyLayer {
+ existsOnAnyLayer = true
+ child.mode = uint32(stat.Mode)
+ child.uid = stat.UID
+ child.gid = stat.GID
+ child.devMajor = stat.DevMajor
+ child.devMinor = stat.DevMinor
+ child.ino = stat.Ino
+ }
+
+ // For non-directory files, only the topmost layer that contains a file
+ // matters.
+ if !isDir {
+ return false
+ }
+
+ // Directories are merged with directories from lower layers if they
+ // are not explicitly opaque.
+ opaqueVal, err := vfsObj.GetxattrAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: childVD,
+ Start: childVD,
+ }, &vfs.GetxattrOptions{
+ Name: _OVL_XATTR_OPAQUE,
+ Size: 1,
+ })
+ return !(err == nil && opaqueVal == "y")
+ })
+
+ if lookupErr != nil {
+ child.destroyLocked()
+ return nil, lookupErr
+ }
+ if !existsOnAnyLayer {
+ child.destroyLocked()
+ return nil, syserror.ENOENT
+ }
+
+ // Device and inode numbers were copied from the topmost layer above;
+ // override them if necessary.
+ if child.isDir() {
+ child.devMajor = linux.UNNAMED_MAJOR
+ child.devMinor = fs.dirDevMinor
+ child.ino = fs.newDirIno()
+ } else if !child.upperVD.Ok() {
+ child.devMajor = linux.UNNAMED_MAJOR
+ child.devMinor = fs.lowerDevMinors[child.lowerVDs[0].Mount().Filesystem()]
+ }
+
+ parent.IncRef()
+ child.parent = parent
+ child.name = name
+ return child, nil
+}
+
+// lookupLayerLocked is similar to lookupLocked, but only returns information
+// about the file rather than a dentry.
+//
+// Preconditions: fs.renameMu must be locked. parent.dirMu must be locked.
+func (fs *filesystem) lookupLayerLocked(ctx context.Context, parent *dentry, name string) (lookupLayer, error) {
+ childPath := fspath.Parse(name)
+ lookupLayer := lookupLayerNone
+ var lookupErr error
+
+ parent.iterLayers(func(parentVD vfs.VirtualDentry, isUpper bool) bool {
+ stat, err := fs.vfsfs.VirtualFilesystem().StatAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: parentVD,
+ Start: parentVD,
+ Path: childPath,
+ }, &vfs.StatOptions{
+ Mask: linux.STATX_TYPE,
+ })
+ if err == syserror.ENOENT || err == syserror.ENAMETOOLONG {
+ // The file doesn't exist on this layer. Proceed to the next
+ // one.
+ return true
+ }
+ if err != nil {
+ lookupErr = err
+ return false
+ }
+ if stat.Mask&linux.STATX_TYPE == 0 {
+ // Linux's overlayfs tends to return EREMOTE in cases where a file
+ // is unusable for reasons that are not better captured by another
+ // errno.
+ lookupErr = syserror.EREMOTE
+ return false
+ }
+ if isWhiteout(&stat) {
+ // This is a whiteout, so it "doesn't exist" on this layer, and
+ // layers below this one are ignored.
+ if isUpper {
+ lookupLayer = lookupLayerUpperWhiteout
+ }
+ return false
+ }
+ // The file exists; we can stop searching.
+ if isUpper {
+ lookupLayer = lookupLayerUpper
+ } else {
+ lookupLayer = lookupLayerLower
+ }
+ return false
+ })
+
+ return lookupLayer, lookupErr
+}
+
+type lookupLayer int
+
+const (
+ // lookupLayerNone indicates that no file exists at the given path on the
+ // upper layer, and is either whited out or does not exist on lower layers.
+ // Therefore, the file does not exist in the overlay filesystem, and file
+ // creation may proceed normally (if an upper layer exists).
+ lookupLayerNone lookupLayer = iota
+
+ // lookupLayerLower indicates that no file exists at the given path on the
+ // upper layer, but exists on a lower layer. Therefore, the file exists in
+ // the overlay filesystem, but must be copied-up before mutation.
+ lookupLayerLower
+
+ // lookupLayerUpper indicates that a non-whiteout file exists at the given
+ // path on the upper layer. Therefore, the file exists in the overlay
+ // filesystem, and is already copied-up.
+ lookupLayerUpper
+
+ // lookupLayerUpperWhiteout indicates that a whiteout exists at the given
+ // path on the upper layer. Therefore, the file does not exist in the
+ // overlay filesystem, and file creation must remove the whiteout before
+ // proceeding.
+ lookupLayerUpperWhiteout
+)
+
+func (ll lookupLayer) existsInOverlay() bool {
+ return ll == lookupLayerLower || ll == lookupLayerUpper
+}
+
+// walkParentDirLocked resolves all but the last path component of rp to an
+// existing directory, starting from the given directory (which is usually
+// rp.Start().Impl().(*dentry)). It does not check that the returned directory
+// is searchable by the provider of rp.
+//
+// Preconditions: fs.renameMu must be locked. !rp.Done().
+func (fs *filesystem) walkParentDirLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, ds **[]*dentry) (*dentry, error) {
+ for !rp.Final() {
+ d.dirMu.Lock()
+ next, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
+ d.dirMu.Unlock()
+ if err != nil {
+ return nil, err
+ }
+ d = next
+ }
+ if !d.isDir() {
+ return nil, syserror.ENOTDIR
+ }
+ return d, nil
+}
+
+// resolveLocked resolves rp to an existing file.
+//
+// Preconditions: fs.renameMu must be locked.
+func (fs *filesystem) resolveLocked(ctx context.Context, rp *vfs.ResolvingPath, ds **[]*dentry) (*dentry, error) {
+ d := rp.Start().Impl().(*dentry)
+ for !rp.Done() {
+ d.dirMu.Lock()
+ next, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
+ d.dirMu.Unlock()
+ if err != nil {
+ return nil, err
+ }
+ d = next
+ }
+ if rp.MustBeDir() && !d.isDir() {
+ return nil, syserror.ENOTDIR
+ }
+ return d, nil
+}
+
+// doCreateAt checks that creating a file at rp is permitted, then invokes
+// create to do so.
+//
+// Preconditions: !rp.Done(). For the final path component in rp,
+// !rp.ShouldFollowSymlink().
+func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir bool, create func(parent *dentry, name string, haveUpperWhiteout bool) error) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ start := rp.Start().Impl().(*dentry)
+ parent, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
+ if err != nil {
+ return err
+ }
+ if err := parent.checkPermissions(rp.Credentials(), vfs.MayWrite|vfs.MayExec); err != nil {
+ return err
+ }
+ name := rp.Component()
+ if name == "." || name == ".." {
+ return syserror.EEXIST
+ }
+ if !dir && rp.MustBeDir() {
+ return syserror.ENOENT
+ }
+ if parent.vfsd.IsDead() {
+ return syserror.ENOENT
+ }
+ mnt := rp.Mount()
+ if err := mnt.CheckBeginWrite(); err != nil {
+ return err
+ }
+ defer mnt.EndWrite()
+ parent.dirMu.Lock()
+ defer parent.dirMu.Unlock()
+
+ // Determine if a file already exists at name.
+ if _, ok := parent.children[name]; ok {
+ return syserror.EEXIST
+ }
+ childLayer, err := fs.lookupLayerLocked(ctx, parent, name)
+ if err != nil {
+ return err
+ }
+ if childLayer.existsInOverlay() {
+ return syserror.EEXIST
+ }
+
+ // Ensure that the parent directory is copied-up so that we can create the
+ // new file in the upper layer.
+ if err := parent.copyUpLocked(ctx); err != nil {
+ return err
+ }
+
+ // Finally create the new file.
+ if err := create(parent, name, childLayer == lookupLayerUpperWhiteout); err != nil {
+ return err
+ }
+ parent.dirents = nil
+ return nil
+}
+
+// Preconditions: pop's parent directory has been copied up.
+func (fs *filesystem) createWhiteout(ctx context.Context, vfsObj *vfs.VirtualFilesystem, pop *vfs.PathOperation) error {
+ return vfsObj.MknodAt(ctx, fs.creds, pop, &vfs.MknodOptions{
+ Mode: linux.S_IFCHR, // permissions == include/linux/fs.h:WHITEOUT_MODE == 0
+ // DevMajor == DevMinor == 0, from include/linux/fs.h:WHITEOUT_DEV
+ })
+}
+
+func (fs *filesystem) cleanupRecreateWhiteout(ctx context.Context, vfsObj *vfs.VirtualFilesystem, pop *vfs.PathOperation) {
+ if err := fs.createWhiteout(ctx, vfsObj, pop); err != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to recreate whiteout after failed file creation: %v", err)
+ }
+}
+
+// AccessAt implements vfs.Filesystem.Impl.AccessAt.
+func (fs *filesystem) AccessAt(ctx context.Context, rp *vfs.ResolvingPath, creds *auth.Credentials, ats vfs.AccessTypes) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return err
+ }
+ return d.checkPermissions(creds, ats)
+}
+
+// BoundEndpointAt implements vfs.FilesystemImpl.BoundEndpointAt.
+func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return nil, err
+ }
+ if err := d.checkPermissions(rp.Credentials(), vfs.MayWrite); err != nil {
+ return nil, err
+ }
+ layerVD := d.topLayer()
+ return fs.vfsfs.VirtualFilesystem().BoundEndpointAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ }, &opts)
+}
+
+// GetDentryAt implements vfs.FilesystemImpl.GetDentryAt.
+func (fs *filesystem) GetDentryAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.GetDentryOptions) (*vfs.Dentry, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return nil, err
+ }
+ if opts.CheckSearchable {
+ if !d.isDir() {
+ return nil, syserror.ENOTDIR
+ }
+ if err := d.checkPermissions(rp.Credentials(), vfs.MayExec); err != nil {
+ return nil, err
+ }
+ }
+ d.IncRef()
+ return &d.vfsd, nil
+}
+
+// GetParentDentryAt implements vfs.FilesystemImpl.GetParentDentryAt.
+func (fs *filesystem) GetParentDentryAt(ctx context.Context, rp *vfs.ResolvingPath) (*vfs.Dentry, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ start := rp.Start().Impl().(*dentry)
+ d, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
+ if err != nil {
+ return nil, err
+ }
+ d.IncRef()
+ return &d.vfsd, nil
+}
+
+// LinkAt implements vfs.FilesystemImpl.LinkAt.
+func (fs *filesystem) LinkAt(ctx context.Context, rp *vfs.ResolvingPath, vd vfs.VirtualDentry) error {
+ return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, childName string, haveUpperWhiteout bool) error {
+ if rp.Mount() != vd.Mount() {
+ return syserror.EXDEV
+ }
+ old := vd.Dentry().Impl().(*dentry)
+ if old.isDir() {
+ return syserror.EPERM
+ }
+ if err := old.copyUpLocked(ctx); err != nil {
+ return err
+ }
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ newpop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(childName),
+ }
+ if haveUpperWhiteout {
+ if err := vfsObj.UnlinkAt(ctx, fs.creds, &newpop); err != nil {
+ return err
+ }
+ }
+ if err := vfsObj.LinkAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: old.upperVD,
+ Start: old.upperVD,
+ }, &newpop); err != nil {
+ if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &newpop)
+ }
+ return err
+ }
+ creds := rp.Credentials()
+ if err := vfsObj.SetStatAt(ctx, fs.creds, &newpop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: uint32(creds.EffectiveKUID),
+ GID: uint32(creds.EffectiveKGID),
+ },
+ }); err != nil {
+ if cleanupErr := vfsObj.UnlinkAt(ctx, fs.creds, &newpop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer file after LinkAt metadata update failure: %v", cleanupErr)
+ } else if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &newpop)
+ }
+ return err
+ }
+ return nil
+ })
+}
+
+// MkdirAt implements vfs.FilesystemImpl.MkdirAt.
+func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.MkdirOptions) error {
+ return fs.doCreateAt(ctx, rp, true /* dir */, func(parent *dentry, childName string, haveUpperWhiteout bool) error {
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ pop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(childName),
+ }
+ if haveUpperWhiteout {
+ if err := vfsObj.UnlinkAt(ctx, fs.creds, &pop); err != nil {
+ return err
+ }
+ }
+ if err := vfsObj.MkdirAt(ctx, fs.creds, &pop, &opts); err != nil {
+ if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ creds := rp.Credentials()
+ if err := vfsObj.SetStatAt(ctx, fs.creds, &pop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: uint32(creds.EffectiveKUID),
+ GID: uint32(creds.EffectiveKGID),
+ },
+ }); err != nil {
+ if cleanupErr := vfsObj.RmdirAt(ctx, fs.creds, &pop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer directory after MkdirAt metadata update failure: %v", cleanupErr)
+ } else if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ if haveUpperWhiteout {
+ // There may be directories on lower layers (previously hidden by
+ // the whiteout) that the new directory should not be merged with.
+ // Mark it opaque to prevent merging.
+ if err := vfsObj.SetxattrAt(ctx, fs.creds, &pop, &vfs.SetxattrOptions{
+ Name: _OVL_XATTR_OPAQUE,
+ Value: "y",
+ }); err != nil {
+ if cleanupErr := vfsObj.RmdirAt(ctx, fs.creds, &pop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer directory after MkdirAt set-opaque failure: %v", cleanupErr)
+ } else {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ }
+ return nil
+ })
+}
+
+// MknodAt implements vfs.FilesystemImpl.MknodAt.
+func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.MknodOptions) error {
+ return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, childName string, haveUpperWhiteout bool) error {
+ // Disallow attempts to create whiteouts.
+ if opts.Mode&linux.S_IFMT == linux.S_IFCHR && opts.DevMajor == 0 && opts.DevMinor == 0 {
+ return syserror.EPERM
+ }
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ pop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(childName),
+ }
+ if haveUpperWhiteout {
+ if err := vfsObj.UnlinkAt(ctx, fs.creds, &pop); err != nil {
+ return err
+ }
+ }
+ if err := vfsObj.MknodAt(ctx, fs.creds, &pop, &opts); err != nil {
+ if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ creds := rp.Credentials()
+ if err := vfsObj.SetStatAt(ctx, fs.creds, &pop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: uint32(creds.EffectiveKUID),
+ GID: uint32(creds.EffectiveKGID),
+ },
+ }); err != nil {
+ if cleanupErr := vfsObj.UnlinkAt(ctx, fs.creds, &pop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer file after MknodAt metadata update failure: %v", cleanupErr)
+ } else if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ return nil
+ })
+}
+
+// OpenAt implements vfs.FilesystemImpl.OpenAt.
+func (fs *filesystem) OpenAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+ mayCreate := opts.Flags&linux.O_CREAT != 0
+ mustCreate := opts.Flags&(linux.O_CREAT|linux.O_EXCL) == (linux.O_CREAT | linux.O_EXCL)
+
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+
+ start := rp.Start().Impl().(*dentry)
+ if rp.Done() {
+ if mustCreate {
+ return nil, syserror.EEXIST
+ }
+ return start.openLocked(ctx, rp, &opts)
+ }
+
+afterTrailingSymlink:
+ parent, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
+ if err != nil {
+ return nil, err
+ }
+ // Check for search permission in the parent directory.
+ if err := parent.checkPermissions(rp.Credentials(), vfs.MayExec); err != nil {
+ return nil, err
+ }
+ // Determine whether or not we need to create a file.
+ parent.dirMu.Lock()
+ child, err := fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
+ if err == syserror.ENOENT && mayCreate {
+ fd, err := fs.createAndOpenLocked(ctx, rp, parent, &opts, &ds)
+ parent.dirMu.Unlock()
+ return fd, err
+ }
+ if err != nil {
+ parent.dirMu.Unlock()
+ return nil, err
+ }
+ // Open existing child or follow symlink.
+ parent.dirMu.Unlock()
+ if mustCreate {
+ return nil, syserror.EEXIST
+ }
+ if child.isSymlink() && rp.ShouldFollowSymlink() {
+ target, err := child.readlink(ctx)
+ if err != nil {
+ return nil, err
+ }
+ if err := rp.HandleSymlink(target); err != nil {
+ return nil, err
+ }
+ start = parent
+ goto afterTrailingSymlink
+ }
+ return child.openLocked(ctx, rp, &opts)
+}
+
+// Preconditions: fs.renameMu must be locked.
+func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vfs.OpenOptions) (*vfs.FileDescription, error) {
+ ats := vfs.AccessTypesForOpenFlags(opts)
+ if err := d.checkPermissions(rp.Credentials(), ats); err != nil {
+ return nil, err
+ }
+ if ats.MayWrite() {
+ if err := d.copyUpLocked(ctx); err != nil {
+ return nil, err
+ }
+ }
+ mnt := rp.Mount()
+
+ // Directory FDs open FDs from each layer when directory entries are read,
+ // so they don't require opening an FD from d.topLayer() up front.
+ ftype := atomic.LoadUint32(&d.mode) & linux.S_IFMT
+ if ftype == linux.S_IFDIR {
+ // Can't open directories with O_CREAT.
+ if opts.Flags&linux.O_CREAT != 0 {
+ return nil, syserror.EISDIR
+ }
+ // Can't open directories writably.
+ if ats&vfs.MayWrite != 0 {
+ return nil, syserror.EISDIR
+ }
+ if opts.Flags&linux.O_DIRECT != 0 {
+ return nil, syserror.EINVAL
+ }
+ fd := &directoryFD{}
+ fd.LockFD.Init(&d.locks)
+ if err := fd.vfsfd.Init(fd, opts.Flags, mnt, &d.vfsd, &vfs.FileDescriptionOptions{
+ UseDentryMetadata: true,
+ }); err != nil {
+ return nil, err
+ }
+ return &fd.vfsfd, nil
+ }
+
+ layerVD, isUpper := d.topLayerInfo()
+ layerFD, err := rp.VirtualFilesystem().OpenAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ }, opts)
+ if err != nil {
+ return nil, err
+ }
+ layerFlags := layerFD.StatusFlags()
+ fd := &nonDirectoryFD{
+ copiedUp: isUpper,
+ cachedFD: layerFD,
+ cachedFlags: layerFlags,
+ }
+ fd.LockFD.Init(&d.locks)
+ layerFDOpts := layerFD.Options()
+ if err := fd.vfsfd.Init(fd, layerFlags, mnt, &d.vfsd, &layerFDOpts); err != nil {
+ layerFD.DecRef()
+ return nil, err
+ }
+ return &fd.vfsfd, nil
+}
+
+// Preconditions: parent.dirMu must be locked. parent does not already contain
+// a child named rp.Component().
+func (fs *filesystem) createAndOpenLocked(ctx context.Context, rp *vfs.ResolvingPath, parent *dentry, opts *vfs.OpenOptions, ds **[]*dentry) (*vfs.FileDescription, error) {
+ creds := rp.Credentials()
+ if err := parent.checkPermissions(creds, vfs.MayWrite); err != nil {
+ return nil, err
+ }
+ if parent.vfsd.IsDead() {
+ return nil, syserror.ENOENT
+ }
+ mnt := rp.Mount()
+ if err := mnt.CheckBeginWrite(); err != nil {
+ return nil, err
+ }
+ defer mnt.EndWrite()
+
+ if err := parent.copyUpLocked(ctx); err != nil {
+ return nil, err
+ }
+
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ childName := rp.Component()
+ pop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(childName),
+ }
+ // We don't know if a whiteout exists on the upper layer; speculatively
+ // unlink it.
+ //
+ // TODO(gvisor.dev/issue/1199): Modify OpenAt => stepLocked so that we do
+ // know whether a whiteout exists.
+ var haveUpperWhiteout bool
+ switch err := vfsObj.UnlinkAt(ctx, fs.creds, &pop); err {
+ case nil:
+ haveUpperWhiteout = true
+ case syserror.ENOENT:
+ haveUpperWhiteout = false
+ default:
+ return nil, err
+ }
+ // Create the file on the upper layer, and get an FD representing it.
+ upperFD, err := vfsObj.OpenAt(ctx, fs.creds, &pop, &vfs.OpenOptions{
+ Flags: opts.Flags&^vfs.FileCreationFlags | linux.O_CREAT | linux.O_EXCL,
+ Mode: opts.Mode,
+ })
+ if err != nil {
+ if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return nil, err
+ }
+ // Change the file's owner to the caller. We can't use upperFD.SetStat()
+ // because it will pick up creds from ctx.
+ if err := vfsObj.SetStatAt(ctx, fs.creds, &pop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: uint32(creds.EffectiveKUID),
+ GID: uint32(creds.EffectiveKGID),
+ },
+ }); err != nil {
+ if cleanupErr := vfsObj.UnlinkAt(ctx, fs.creds, &pop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer file after OpenAt(O_CREAT) metadata update failure: %v", cleanupErr)
+ } else if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return nil, err
+ }
+ // Re-lookup to get a dentry representing the new file, which is needed for
+ // the returned FD.
+ child, err := fs.getChildLocked(ctx, parent, childName, ds)
+ if err != nil {
+ if cleanupErr := vfsObj.UnlinkAt(ctx, fs.creds, &pop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer file after OpenAt(O_CREAT) dentry lookup failure: %v", cleanupErr)
+ } else if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return nil, err
+ }
+ // Finally construct the overlay FD.
+ upperFlags := upperFD.StatusFlags()
+ fd := &nonDirectoryFD{
+ copiedUp: true,
+ cachedFD: upperFD,
+ cachedFlags: upperFlags,
+ }
+ fd.LockFD.Init(&child.locks)
+ upperFDOpts := upperFD.Options()
+ if err := fd.vfsfd.Init(fd, upperFlags, mnt, &child.vfsd, &upperFDOpts); err != nil {
+ upperFD.DecRef()
+ // Don't bother with cleanup; the file was created successfully, we
+ // just can't open it anymore for some reason.
+ return nil, err
+ }
+ return &fd.vfsfd, nil
+}
+
+// ReadlinkAt implements vfs.FilesystemImpl.ReadlinkAt.
+func (fs *filesystem) ReadlinkAt(ctx context.Context, rp *vfs.ResolvingPath) (string, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return "", err
+ }
+ layerVD := d.topLayer()
+ return fs.vfsfs.VirtualFilesystem().ReadlinkAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ })
+}
+
+// RenameAt implements vfs.FilesystemImpl.RenameAt.
+func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldParentVD vfs.VirtualDentry, oldName string, opts vfs.RenameOptions) error {
+ if opts.Flags != 0 {
+ return syserror.EINVAL
+ }
+
+ var ds *[]*dentry
+ fs.renameMu.Lock()
+ defer fs.renameMuUnlockAndCheckDrop(&ds)
+ newParent, err := fs.walkParentDirLocked(ctx, rp, rp.Start().Impl().(*dentry), &ds)
+ if err != nil {
+ return err
+ }
+ newName := rp.Component()
+ if newName == "." || newName == ".." {
+ return syserror.EBUSY
+ }
+ mnt := rp.Mount()
+ if mnt != oldParentVD.Mount() {
+ return syserror.EXDEV
+ }
+ if err := mnt.CheckBeginWrite(); err != nil {
+ return err
+ }
+ defer mnt.EndWrite()
+
+ // FIXME(gvisor.dev/issue/1199): Actually implement rename.
+ _ = newParent
+ return syserror.EXDEV
+}
+
+// RmdirAt implements vfs.FilesystemImpl.RmdirAt.
+func (fs *filesystem) RmdirAt(ctx context.Context, rp *vfs.ResolvingPath) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ start := rp.Start().Impl().(*dentry)
+ parent, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
+ if err != nil {
+ return err
+ }
+ if err := parent.checkPermissions(rp.Credentials(), vfs.MayWrite|vfs.MayExec); err != nil {
+ return err
+ }
+ if err := rp.Mount().CheckBeginWrite(); err != nil {
+ return err
+ }
+ defer rp.Mount().EndWrite()
+ name := rp.Component()
+ if name == "." {
+ return syserror.EINVAL
+ }
+ if name == ".." {
+ return syserror.ENOTEMPTY
+ }
+ vfsObj := rp.VirtualFilesystem()
+ mntns := vfs.MountNamespaceFromContext(ctx)
+ defer mntns.DecRef()
+ parent.dirMu.Lock()
+ defer parent.dirMu.Unlock()
+
+ // Ensure that parent is copied-up before potentially holding child.copyMu
+ // below.
+ if err := parent.copyUpLocked(ctx); err != nil {
+ return err
+ }
+
+ // Unlike UnlinkAt, we need a dentry representing the child directory being
+ // removed in order to verify that it's empty.
+ child, err := fs.getChildLocked(ctx, parent, name, &ds)
+ if err != nil {
+ return err
+ }
+ if !child.isDir() {
+ return syserror.ENOTDIR
+ }
+ child.dirMu.Lock()
+ defer child.dirMu.Unlock()
+ whiteouts, err := child.collectWhiteoutsForRmdirLocked(ctx)
+ if err != nil {
+ return err
+ }
+ child.copyMu.RLock()
+ defer child.copyMu.RUnlock()
+ if err := vfsObj.PrepareDeleteDentry(mntns, &child.vfsd); err != nil {
+ return err
+ }
+
+ pop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(name),
+ }
+ if child.upperVD.Ok() {
+ cleanupRecreateWhiteouts := func() {
+ if !child.upperVD.Ok() {
+ return
+ }
+ for whiteoutName, whiteoutUpper := range whiteouts {
+ if !whiteoutUpper {
+ continue
+ }
+ if err := fs.createWhiteout(ctx, vfsObj, &vfs.PathOperation{
+ Root: child.upperVD,
+ Start: child.upperVD,
+ Path: fspath.Parse(whiteoutName),
+ }); err != nil && err != syserror.EEXIST {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to recreate deleted whiteout after RmdirAt failure: %v", err)
+ }
+ }
+ }
+ // Remove existing whiteouts on the upper layer.
+ for whiteoutName, whiteoutUpper := range whiteouts {
+ if !whiteoutUpper {
+ continue
+ }
+ if err := vfsObj.UnlinkAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: child.upperVD,
+ Start: child.upperVD,
+ Path: fspath.Parse(whiteoutName),
+ }); err != nil {
+ cleanupRecreateWhiteouts()
+ vfsObj.AbortDeleteDentry(&child.vfsd)
+ return err
+ }
+ }
+ // Remove the existing directory on the upper layer.
+ if err := vfsObj.RmdirAt(ctx, fs.creds, &pop); err != nil {
+ cleanupRecreateWhiteouts()
+ vfsObj.AbortDeleteDentry(&child.vfsd)
+ return err
+ }
+ }
+ if err := fs.createWhiteout(ctx, vfsObj, &pop); err != nil {
+ // Don't attempt to recover from this: the original directory is
+ // already gone, so any dentries representing it are invalid, and
+ // creating a new directory won't undo that.
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to create whiteout during RmdirAt: %v", err)
+ vfsObj.AbortDeleteDentry(&child.vfsd)
+ return err
+ }
+
+ vfsObj.CommitDeleteDentry(&child.vfsd)
+ delete(parent.children, name)
+ ds = appendDentry(ds, child)
+ parent.dirents = nil
+ return nil
+}
+
+// SetStatAt implements vfs.FilesystemImpl.SetStatAt.
+func (fs *filesystem) SetStatAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.SetStatOptions) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return err
+ }
+
+ mode := linux.FileMode(atomic.LoadUint32(&d.mode))
+ if err := vfs.CheckSetStat(ctx, rp.Credentials(), &opts.Stat, mode, auth.KUID(atomic.LoadUint32(&d.uid)), auth.KGID(atomic.LoadUint32(&d.gid))); err != nil {
+ return err
+ }
+ mnt := rp.Mount()
+ if err := mnt.CheckBeginWrite(); err != nil {
+ return err
+ }
+ defer mnt.EndWrite()
+ if err := d.copyUpLocked(ctx); err != nil {
+ return err
+ }
+ // Changes to d's attributes are serialized by d.copyMu.
+ d.copyMu.Lock()
+ defer d.copyMu.Unlock()
+ if err := d.fs.vfsfs.VirtualFilesystem().SetStatAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: d.upperVD,
+ Start: d.upperVD,
+ }, &opts); err != nil {
+ return err
+ }
+ d.updateAfterSetStatLocked(&opts)
+ return nil
+}
+
+// StatAt implements vfs.FilesystemImpl.StatAt.
+func (fs *filesystem) StatAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.StatOptions) (linux.Statx, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ d, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return linux.Statx{}, err
+ }
+
+ var stat linux.Statx
+ if layerMask := opts.Mask &^ statInternalMask; layerMask != 0 {
+ layerVD := d.topLayer()
+ stat, err = fs.vfsfs.VirtualFilesystem().StatAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ }, &vfs.StatOptions{
+ Mask: layerMask,
+ Sync: opts.Sync,
+ })
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ }
+ d.statInternalTo(ctx, &opts, &stat)
+ return stat, nil
+}
+
+// StatFSAt implements vfs.FilesystemImpl.StatFSAt.
+func (fs *filesystem) StatFSAt(ctx context.Context, rp *vfs.ResolvingPath) (linux.Statfs, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ _, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return linux.Statfs{}, err
+ }
+ return fs.statFS(ctx)
+}
+
+// SymlinkAt implements vfs.FilesystemImpl.SymlinkAt.
+func (fs *filesystem) SymlinkAt(ctx context.Context, rp *vfs.ResolvingPath, target string) error {
+ return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, childName string, haveUpperWhiteout bool) error {
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ pop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(childName),
+ }
+ if haveUpperWhiteout {
+ if err := vfsObj.UnlinkAt(ctx, fs.creds, &pop); err != nil {
+ return err
+ }
+ }
+ if err := vfsObj.SymlinkAt(ctx, fs.creds, &pop, target); err != nil {
+ if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ creds := rp.Credentials()
+ if err := vfsObj.SetStatAt(ctx, fs.creds, &pop, &vfs.SetStatOptions{
+ Stat: linux.Statx{
+ Mask: linux.STATX_UID | linux.STATX_GID,
+ UID: uint32(creds.EffectiveKUID),
+ GID: uint32(creds.EffectiveKGID),
+ },
+ }); err != nil {
+ if cleanupErr := vfsObj.UnlinkAt(ctx, fs.creds, &pop); cleanupErr != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to delete upper layer file after SymlinkAt metadata update failure: %v", cleanupErr)
+ } else if haveUpperWhiteout {
+ fs.cleanupRecreateWhiteout(ctx, vfsObj, &pop)
+ }
+ return err
+ }
+ return nil
+ })
+}
+
+// UnlinkAt implements vfs.FilesystemImpl.UnlinkAt.
+func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ start := rp.Start().Impl().(*dentry)
+ parent, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
+ if err != nil {
+ return err
+ }
+ if err := parent.checkPermissions(rp.Credentials(), vfs.MayWrite|vfs.MayExec); err != nil {
+ return err
+ }
+ if err := rp.Mount().CheckBeginWrite(); err != nil {
+ return err
+ }
+ defer rp.Mount().EndWrite()
+ name := rp.Component()
+ if name == "." || name == ".." {
+ return syserror.EISDIR
+ }
+ if rp.MustBeDir() {
+ return syserror.ENOTDIR
+ }
+ vfsObj := rp.VirtualFilesystem()
+ mntns := vfs.MountNamespaceFromContext(ctx)
+ defer mntns.DecRef()
+ parent.dirMu.Lock()
+ defer parent.dirMu.Unlock()
+
+ // Ensure that parent is copied-up before potentially holding child.copyMu
+ // below.
+ if err := parent.copyUpLocked(ctx); err != nil {
+ return err
+ }
+
+ child := parent.children[name]
+ var childLayer lookupLayer
+ if child != nil {
+ if child.isDir() {
+ return syserror.EISDIR
+ }
+ if err := vfsObj.PrepareDeleteDentry(mntns, &child.vfsd); err != nil {
+ return err
+ }
+ // Hold child.copyMu to prevent it from being copied-up during
+ // deletion.
+ child.copyMu.RLock()
+ defer child.copyMu.RUnlock()
+ if child.upperVD.Ok() {
+ childLayer = lookupLayerUpper
+ } else {
+ childLayer = lookupLayerLower
+ }
+ } else {
+ // Determine if the file being unlinked actually exists. Holding
+ // parent.dirMu prevents a dentry from being instantiated for the file,
+ // which in turn prevents it from being copied-up, so this result is
+ // stable.
+ childLayer, err = fs.lookupLayerLocked(ctx, parent, name)
+ if err != nil {
+ return err
+ }
+ if !childLayer.existsInOverlay() {
+ return syserror.ENOENT
+ }
+ }
+
+ pop := vfs.PathOperation{
+ Root: parent.upperVD,
+ Start: parent.upperVD,
+ Path: fspath.Parse(name),
+ }
+ if childLayer == lookupLayerUpper {
+ // Remove the existing file on the upper layer.
+ if err := vfsObj.UnlinkAt(ctx, fs.creds, &pop); err != nil {
+ if child != nil {
+ vfsObj.AbortDeleteDentry(&child.vfsd)
+ }
+ return err
+ }
+ }
+ if err := fs.createWhiteout(ctx, vfsObj, &pop); err != nil {
+ ctx.Warningf("Unrecoverable overlayfs inconsistency: failed to create whiteout during UnlinkAt: %v", err)
+ if child != nil {
+ vfsObj.AbortDeleteDentry(&child.vfsd)
+ }
+ return err
+ }
+
+ if child != nil {
+ vfsObj.CommitDeleteDentry(&child.vfsd)
+ delete(parent.children, name)
+ ds = appendDentry(ds, child)
+ }
+ parent.dirents = nil
+ return nil
+}
+
+// ListxattrAt implements vfs.FilesystemImpl.ListxattrAt.
+func (fs *filesystem) ListxattrAt(ctx context.Context, rp *vfs.ResolvingPath, size uint64) ([]string, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ _, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return nil, err
+ }
+ // TODO(gvisor.dev/issue/1199): Linux overlayfs actually allows listxattr,
+ // but not any other xattr syscalls. For now we just reject all of them.
+ return nil, syserror.ENOTSUP
+}
+
+// GetxattrAt implements vfs.FilesystemImpl.GetxattrAt.
+func (fs *filesystem) GetxattrAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.GetxattrOptions) (string, error) {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ _, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return "", err
+ }
+ return "", syserror.ENOTSUP
+}
+
+// SetxattrAt implements vfs.FilesystemImpl.SetxattrAt.
+func (fs *filesystem) SetxattrAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.SetxattrOptions) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ _, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return err
+ }
+ return syserror.ENOTSUP
+}
+
+// RemovexattrAt implements vfs.FilesystemImpl.RemovexattrAt.
+func (fs *filesystem) RemovexattrAt(ctx context.Context, rp *vfs.ResolvingPath, name string) error {
+ var ds *[]*dentry
+ fs.renameMu.RLock()
+ defer fs.renameMuRUnlockAndCheckDrop(&ds)
+ _, err := fs.resolveLocked(ctx, rp, &ds)
+ if err != nil {
+ return err
+ }
+ return syserror.ENOTSUP
+}
+
+// PrependPath implements vfs.FilesystemImpl.PrependPath.
+func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDentry, b *fspath.Builder) error {
+ fs.renameMu.RLock()
+ defer fs.renameMu.RUnlock()
+ return genericPrependPath(vfsroot, vd.Mount(), vd.Dentry().Impl().(*dentry), b)
+}
diff --git a/pkg/sentry/fsimpl/overlay/non_directory.go b/pkg/sentry/fsimpl/overlay/non_directory.go
new file mode 100644
index 000000000..a3c1f7a8d
--- /dev/null
+++ b/pkg/sentry/fsimpl/overlay/non_directory.go
@@ -0,0 +1,266 @@
+// Copyright 2020 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 overlay
+
+import (
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/memmap"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+func (d *dentry) isSymlink() bool {
+ return atomic.LoadUint32(&d.mode)&linux.S_IFMT == linux.S_IFLNK
+}
+
+func (d *dentry) readlink(ctx context.Context) (string, error) {
+ layerVD := d.topLayer()
+ return d.fs.vfsfs.VirtualFilesystem().ReadlinkAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: layerVD,
+ Start: layerVD,
+ })
+}
+
+type nonDirectoryFD struct {
+ fileDescription
+
+ // If copiedUp is false, cachedFD represents
+ // fileDescription.dentry().lowerVDs[0]; otherwise, cachedFD represents
+ // fileDescription.dentry().upperVD. cachedFlags is the last known value of
+ // cachedFD.StatusFlags(). copiedUp, cachedFD, and cachedFlags are
+ // protected by mu.
+ mu sync.Mutex
+ copiedUp bool
+ cachedFD *vfs.FileDescription
+ cachedFlags uint32
+}
+
+func (fd *nonDirectoryFD) getCurrentFD(ctx context.Context) (*vfs.FileDescription, error) {
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
+ wrappedFD, err := fd.currentFDLocked(ctx)
+ if err != nil {
+ return nil, err
+ }
+ wrappedFD.IncRef()
+ return wrappedFD, nil
+}
+
+func (fd *nonDirectoryFD) currentFDLocked(ctx context.Context) (*vfs.FileDescription, error) {
+ d := fd.dentry()
+ statusFlags := fd.vfsfd.StatusFlags()
+ if !fd.copiedUp && d.isCopiedUp() {
+ // Switch to the copied-up file.
+ upperVD := d.topLayer()
+ upperFD, err := fd.filesystem().vfsfs.VirtualFilesystem().OpenAt(ctx, d.fs.creds, &vfs.PathOperation{
+ Root: upperVD,
+ Start: upperVD,
+ }, &vfs.OpenOptions{
+ Flags: statusFlags,
+ })
+ if err != nil {
+ return nil, err
+ }
+ oldOff, oldOffErr := fd.cachedFD.Seek(ctx, 0, linux.SEEK_CUR)
+ if oldOffErr == nil {
+ if _, err := upperFD.Seek(ctx, oldOff, linux.SEEK_SET); err != nil {
+ upperFD.DecRef()
+ return nil, err
+ }
+ }
+ fd.cachedFD.DecRef()
+ fd.copiedUp = true
+ fd.cachedFD = upperFD
+ fd.cachedFlags = statusFlags
+ } else if fd.cachedFlags != statusFlags {
+ if err := fd.cachedFD.SetStatusFlags(ctx, d.fs.creds, statusFlags); err != nil {
+ return nil, err
+ }
+ fd.cachedFlags = statusFlags
+ }
+ return fd.cachedFD, nil
+}
+
+// Release implements vfs.FileDescriptionImpl.Release.
+func (fd *nonDirectoryFD) Release() {
+ fd.cachedFD.DecRef()
+ fd.cachedFD = nil
+}
+
+// OnClose implements vfs.FileDescriptionImpl.OnClose.
+func (fd *nonDirectoryFD) OnClose(ctx context.Context) error {
+ // Linux doesn't define ovl_file_operations.flush at all (i.e. its
+ // equivalent to OnClose is a no-op). We pass through to
+ // fd.cachedFD.OnClose() without upgrading if fd.dentry() has been
+ // copied-up, since OnClose is mostly used to define post-close writeback,
+ // and if fd.cachedFD hasn't been updated then it can't have been used to
+ // mutate fd.dentry() anyway.
+ fd.mu.Lock()
+ if statusFlags := fd.vfsfd.StatusFlags(); fd.cachedFlags != statusFlags {
+ if err := fd.cachedFD.SetStatusFlags(ctx, fd.filesystem().creds, statusFlags); err != nil {
+ fd.mu.Unlock()
+ return err
+ }
+ fd.cachedFlags = statusFlags
+ }
+ wrappedFD := fd.cachedFD
+ defer wrappedFD.IncRef()
+ fd.mu.Unlock()
+ return wrappedFD.OnClose(ctx)
+}
+
+// Stat implements vfs.FileDescriptionImpl.Stat.
+func (fd *nonDirectoryFD) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) {
+ var stat linux.Statx
+ if layerMask := opts.Mask &^ statInternalMask; layerMask != 0 {
+ wrappedFD, err := fd.getCurrentFD(ctx)
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ stat, err = wrappedFD.Stat(ctx, vfs.StatOptions{
+ Mask: layerMask,
+ Sync: opts.Sync,
+ })
+ wrappedFD.DecRef()
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ }
+ fd.dentry().statInternalTo(ctx, &opts, &stat)
+ return stat, nil
+}
+
+// SetStat implements vfs.FileDescriptionImpl.SetStat.
+func (fd *nonDirectoryFD) SetStat(ctx context.Context, opts vfs.SetStatOptions) error {
+ d := fd.dentry()
+ mode := linux.FileMode(atomic.LoadUint32(&d.mode))
+ if err := vfs.CheckSetStat(ctx, auth.CredentialsFromContext(ctx), &opts.Stat, mode, auth.KUID(atomic.LoadUint32(&d.uid)), auth.KGID(atomic.LoadUint32(&d.gid))); err != nil {
+ return err
+ }
+ mnt := fd.vfsfd.Mount()
+ if err := mnt.CheckBeginWrite(); err != nil {
+ return err
+ }
+ defer mnt.EndWrite()
+ if err := d.copyUpLocked(ctx); err != nil {
+ return err
+ }
+ // Changes to d's attributes are serialized by d.copyMu.
+ d.copyMu.Lock()
+ defer d.copyMu.Unlock()
+ wrappedFD, err := fd.currentFDLocked(ctx)
+ if err != nil {
+ return err
+ }
+ if err := wrappedFD.SetStat(ctx, opts); err != nil {
+ return err
+ }
+ d.updateAfterSetStatLocked(&opts)
+ return nil
+}
+
+// StatFS implements vfs.FileDesciptionImpl.StatFS.
+func (fd *nonDirectoryFD) StatFS(ctx context.Context) (linux.Statfs, error) {
+ return fd.filesystem().statFS(ctx)
+}
+
+// PRead implements vfs.FileDescriptionImpl.PRead.
+func (fd *nonDirectoryFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
+ wrappedFD, err := fd.getCurrentFD(ctx)
+ if err != nil {
+ return 0, err
+ }
+ defer wrappedFD.DecRef()
+ return wrappedFD.PRead(ctx, dst, offset, opts)
+}
+
+// Read implements vfs.FileDescriptionImpl.Read.
+func (fd *nonDirectoryFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+ // Hold fd.mu during the read to serialize the file offset.
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
+ wrappedFD, err := fd.currentFDLocked(ctx)
+ if err != nil {
+ return 0, err
+ }
+ return wrappedFD.Read(ctx, dst, opts)
+}
+
+// PWrite implements vfs.FileDescriptionImpl.PWrite.
+func (fd *nonDirectoryFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
+ wrappedFD, err := fd.getCurrentFD(ctx)
+ if err != nil {
+ return 0, err
+ }
+ defer wrappedFD.DecRef()
+ return wrappedFD.PWrite(ctx, src, offset, opts)
+}
+
+// Write implements vfs.FileDescriptionImpl.Write.
+func (fd *nonDirectoryFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+ // Hold fd.mu during the write to serialize the file offset.
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
+ wrappedFD, err := fd.currentFDLocked(ctx)
+ if err != nil {
+ return 0, err
+ }
+ return wrappedFD.Write(ctx, src, opts)
+}
+
+// Seek implements vfs.FileDescriptionImpl.Seek.
+func (fd *nonDirectoryFD) Seek(ctx context.Context, offset int64, whence int32) (int64, error) {
+ // Hold fd.mu during the seek to serialize the file offset.
+ fd.mu.Lock()
+ defer fd.mu.Unlock()
+ wrappedFD, err := fd.currentFDLocked(ctx)
+ if err != nil {
+ return 0, err
+ }
+ return wrappedFD.Seek(ctx, offset, whence)
+}
+
+// Sync implements vfs.FileDescriptionImpl.Sync.
+func (fd *nonDirectoryFD) Sync(ctx context.Context) error {
+ fd.mu.Lock()
+ if !fd.dentry().isCopiedUp() {
+ fd.mu.Unlock()
+ return nil
+ }
+ wrappedFD, err := fd.currentFDLocked(ctx)
+ if err != nil {
+ fd.mu.Unlock()
+ return err
+ }
+ wrappedFD.IncRef()
+ defer wrappedFD.DecRef()
+ fd.mu.Unlock()
+ return wrappedFD.Sync(ctx)
+}
+
+// ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap.
+func (fd *nonDirectoryFD) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error {
+ wrappedFD, err := fd.getCurrentFD(ctx)
+ if err != nil {
+ return err
+ }
+ defer wrappedFD.DecRef()
+ return wrappedFD.ConfigureMMap(ctx, opts)
+}
diff --git a/pkg/sentry/fsimpl/overlay/overlay.go b/pkg/sentry/fsimpl/overlay/overlay.go
new file mode 100644
index 000000000..e660d0e2c
--- /dev/null
+++ b/pkg/sentry/fsimpl/overlay/overlay.go
@@ -0,0 +1,612 @@
+// Copyright 2020 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 overlay provides an overlay filesystem implementation, which
+// synthesizes a filesystem by composing one or more immutable filesystems
+// ("lower layers") with an optional mutable filesystem ("upper layer").
+//
+// Lock order:
+//
+// directoryFD.mu / nonDirectoryFD.mu
+// filesystem.renameMu
+// dentry.dirMu
+// dentry.copyMu
+//
+// Locking dentry.dirMu in multiple dentries requires that parent dentries are
+// locked before child dentries, and that filesystem.renameMu is locked to
+// stabilize this relationship.
+package overlay
+
+import (
+ "strings"
+ "sync/atomic"
+
+ "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/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sentry/vfs/lock"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+// Name is the default filesystem name.
+const Name = "overlay"
+
+// FilesystemType implements vfs.FilesystemType.
+type FilesystemType struct{}
+
+// Name implements vfs.FilesystemType.Name.
+func (FilesystemType) Name() string {
+ return Name
+}
+
+// FilesystemOptions may be passed as vfs.GetFilesystemOptions.InternalData to
+// FilesystemType.GetFilesystem.
+type FilesystemOptions struct {
+ // Callers passing FilesystemOptions to
+ // overlay.FilesystemType.GetFilesystem() are responsible for ensuring that
+ // the vfs.Mounts comprising the layers of the overlay filesystem do not
+ // contain submounts.
+
+ // If UpperRoot.Ok(), it is the root of the writable upper layer of the
+ // overlay.
+ UpperRoot vfs.VirtualDentry
+
+ // LowerRoots contains the roots of the immutable lower layers of the
+ // overlay. LowerRoots is immutable.
+ LowerRoots []vfs.VirtualDentry
+}
+
+// filesystem implements vfs.FilesystemImpl.
+type filesystem struct {
+ vfsfs vfs.Filesystem
+
+ // Immutable options.
+ opts FilesystemOptions
+
+ // creds is a copy of the filesystem's creator's credentials, which are
+ // used for accesses to the filesystem's layers. creds is immutable.
+ creds *auth.Credentials
+
+ // dirDevMinor is the device minor number used for directories. dirDevMinor
+ // is immutable.
+ dirDevMinor uint32
+
+ // lowerDevMinors maps lower layer filesystems to device minor numbers
+ // assigned to non-directory files originating from that filesystem.
+ // lowerDevMinors is immutable.
+ lowerDevMinors map[*vfs.Filesystem]uint32
+
+ // renameMu synchronizes renaming with non-renaming operations in order to
+ // ensure consistent lock ordering between dentry.dirMu in different
+ // dentries.
+ renameMu sync.RWMutex
+
+ // lastDirIno is the last inode number assigned to a directory. lastDirIno
+ // is accessed using atomic memory operations.
+ lastDirIno uint64
+}
+
+// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
+func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
+ mopts := vfs.GenericParseMountOptions(opts.Data)
+ fsoptsRaw := opts.InternalData
+ fsopts, haveFSOpts := fsoptsRaw.(FilesystemOptions)
+ if fsoptsRaw != nil && !haveFSOpts {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: GetFilesystemOptions.InternalData has type %T, wanted overlay.FilesystemOptions or nil", fsoptsRaw)
+ return nil, nil, syserror.EINVAL
+ }
+ if haveFSOpts {
+ if len(fsopts.LowerRoots) == 0 {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: LowerRoots must be non-empty")
+ return nil, nil, syserror.EINVAL
+ }
+ if len(fsopts.LowerRoots) < 2 && !fsopts.UpperRoot.Ok() {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: at least two LowerRoots are required when UpperRoot is unspecified")
+ return nil, nil, syserror.EINVAL
+ }
+ // We don't enforce a maximum number of lower layers when not
+ // configured by applications; the sandbox owner can have an overlay
+ // filesystem with any number of lower layers.
+ } else {
+ vfsroot := vfs.RootFromContext(ctx)
+ defer vfsroot.DecRef()
+ upperPathname, ok := mopts["upperdir"]
+ if ok {
+ delete(mopts, "upperdir")
+ // Linux overlayfs also requires a workdir when upperdir is
+ // specified; we don't, so silently ignore this option.
+ delete(mopts, "workdir")
+ upperPath := fspath.Parse(upperPathname)
+ if !upperPath.Absolute {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: upperdir %q must be absolute", upperPathname)
+ return nil, nil, syserror.EINVAL
+ }
+ upperRoot, err := vfsObj.GetDentryAt(ctx, creds, &vfs.PathOperation{
+ Root: vfsroot,
+ Start: vfsroot,
+ Path: upperPath,
+ FollowFinalSymlink: true,
+ }, &vfs.GetDentryOptions{
+ CheckSearchable: true,
+ })
+ if err != nil {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: failed to resolve upperdir %q: %v", upperPathname, err)
+ return nil, nil, err
+ }
+ defer upperRoot.DecRef()
+ privateUpperRoot, err := clonePrivateMount(vfsObj, upperRoot, false /* forceReadOnly */)
+ if err != nil {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: failed to make private bind mount of upperdir %q: %v", upperPathname, err)
+ return nil, nil, err
+ }
+ defer privateUpperRoot.DecRef()
+ fsopts.UpperRoot = privateUpperRoot
+ }
+ lowerPathnamesStr, ok := mopts["lowerdir"]
+ if !ok {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: missing required option lowerdir")
+ return nil, nil, syserror.EINVAL
+ }
+ delete(mopts, "lowerdir")
+ lowerPathnames := strings.Split(lowerPathnamesStr, ":")
+ const maxLowerLayers = 500 // Linux: fs/overlay/super.c:OVL_MAX_STACK
+ if len(lowerPathnames) < 2 && !fsopts.UpperRoot.Ok() {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: at least two lowerdirs are required when upperdir is unspecified")
+ return nil, nil, syserror.EINVAL
+ }
+ if len(lowerPathnames) > maxLowerLayers {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: %d lowerdirs specified, maximum %d", len(lowerPathnames), maxLowerLayers)
+ return nil, nil, syserror.EINVAL
+ }
+ for _, lowerPathname := range lowerPathnames {
+ lowerPath := fspath.Parse(lowerPathname)
+ if !lowerPath.Absolute {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: lowerdir %q must be absolute", lowerPathname)
+ return nil, nil, syserror.EINVAL
+ }
+ lowerRoot, err := vfsObj.GetDentryAt(ctx, creds, &vfs.PathOperation{
+ Root: vfsroot,
+ Start: vfsroot,
+ Path: lowerPath,
+ FollowFinalSymlink: true,
+ }, &vfs.GetDentryOptions{
+ CheckSearchable: true,
+ })
+ if err != nil {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: failed to resolve lowerdir %q: %v", lowerPathname, err)
+ return nil, nil, err
+ }
+ defer lowerRoot.DecRef()
+ privateLowerRoot, err := clonePrivateMount(vfsObj, lowerRoot, true /* forceReadOnly */)
+ if err != nil {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: failed to make private bind mount of lowerdir %q: %v", lowerPathname, err)
+ return nil, nil, err
+ }
+ defer privateLowerRoot.DecRef()
+ fsopts.LowerRoots = append(fsopts.LowerRoots, privateLowerRoot)
+ }
+ }
+ if len(mopts) != 0 {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: unused options: %v", mopts)
+ return nil, nil, syserror.EINVAL
+ }
+
+ // Allocate device numbers.
+ dirDevMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+ lowerDevMinors := make(map[*vfs.Filesystem]uint32)
+ for _, lowerRoot := range fsopts.LowerRoots {
+ lowerFS := lowerRoot.Mount().Filesystem()
+ if _, ok := lowerDevMinors[lowerFS]; !ok {
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ vfsObj.PutAnonBlockDevMinor(dirDevMinor)
+ for _, lowerDevMinor := range lowerDevMinors {
+ vfsObj.PutAnonBlockDevMinor(lowerDevMinor)
+ }
+ return nil, nil, err
+ }
+ lowerDevMinors[lowerFS] = devMinor
+ }
+ }
+
+ // Take extra references held by the filesystem.
+ if fsopts.UpperRoot.Ok() {
+ fsopts.UpperRoot.IncRef()
+ }
+ for _, lowerRoot := range fsopts.LowerRoots {
+ lowerRoot.IncRef()
+ }
+
+ fs := &filesystem{
+ opts: fsopts,
+ creds: creds.Fork(),
+ dirDevMinor: dirDevMinor,
+ lowerDevMinors: lowerDevMinors,
+ }
+ fs.vfsfs.Init(vfsObj, &fstype, fs)
+
+ // Construct the root dentry.
+ root := fs.newDentry()
+ root.refs = 1
+ if fs.opts.UpperRoot.Ok() {
+ fs.opts.UpperRoot.IncRef()
+ root.copiedUp = 1
+ root.upperVD = fs.opts.UpperRoot
+ }
+ for _, lowerRoot := range fs.opts.LowerRoots {
+ lowerRoot.IncRef()
+ root.lowerVDs = append(root.lowerVDs, lowerRoot)
+ }
+ rootTopVD := root.topLayer()
+ // Get metadata from the topmost layer. See fs.lookupLocked().
+ const rootStatMask = linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID | linux.STATX_INO
+ rootStat, err := vfsObj.StatAt(ctx, creds, &vfs.PathOperation{
+ Root: rootTopVD,
+ Start: rootTopVD,
+ }, &vfs.StatOptions{
+ Mask: rootStatMask,
+ })
+ if err != nil {
+ root.destroyLocked()
+ fs.vfsfs.DecRef()
+ return nil, nil, err
+ }
+ if rootStat.Mask&rootStatMask != rootStatMask {
+ root.destroyLocked()
+ fs.vfsfs.DecRef()
+ return nil, nil, syserror.EREMOTE
+ }
+ if isWhiteout(&rootStat) {
+ ctx.Warningf("overlay.FilesystemType.GetFilesystem: filesystem root is a whiteout")
+ root.destroyLocked()
+ fs.vfsfs.DecRef()
+ return nil, nil, syserror.EINVAL
+ }
+ root.mode = uint32(rootStat.Mode)
+ root.uid = rootStat.UID
+ root.gid = rootStat.GID
+ if rootStat.Mode&linux.S_IFMT == linux.S_IFDIR {
+ root.devMajor = linux.UNNAMED_MAJOR
+ root.devMinor = fs.dirDevMinor
+ root.ino = fs.newDirIno()
+ } else if !root.upperVD.Ok() {
+ root.devMajor = linux.UNNAMED_MAJOR
+ root.devMinor = fs.lowerDevMinors[root.lowerVDs[0].Mount().Filesystem()]
+ root.ino = rootStat.Ino
+ } else {
+ root.devMajor = rootStat.DevMajor
+ root.devMinor = rootStat.DevMinor
+ root.ino = rootStat.Ino
+ }
+
+ return &fs.vfsfs, &root.vfsd, nil
+}
+
+// clonePrivateMount creates a non-recursive bind mount rooted at vd, not
+// associated with any MountNamespace, and returns the root of the new mount.
+// (This is required to ensure that each layer of an overlay comprises only a
+// single mount, and therefore can't cross into e.g. the overlay filesystem
+// itself, risking lock recursion.) A reference is held on the returned
+// VirtualDentry.
+func clonePrivateMount(vfsObj *vfs.VirtualFilesystem, vd vfs.VirtualDentry, forceReadOnly bool) (vfs.VirtualDentry, error) {
+ oldmnt := vd.Mount()
+ opts := oldmnt.Options()
+ if forceReadOnly {
+ opts.ReadOnly = true
+ }
+ newmnt, err := vfsObj.NewDisconnectedMount(oldmnt.Filesystem(), vd.Dentry(), &opts)
+ if err != nil {
+ return vfs.VirtualDentry{}, err
+ }
+ return vfs.MakeVirtualDentry(newmnt, vd.Dentry()), nil
+}
+
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release() {
+ vfsObj := fs.vfsfs.VirtualFilesystem()
+ vfsObj.PutAnonBlockDevMinor(fs.dirDevMinor)
+ for _, lowerDevMinor := range fs.lowerDevMinors {
+ vfsObj.PutAnonBlockDevMinor(lowerDevMinor)
+ }
+ if fs.opts.UpperRoot.Ok() {
+ fs.opts.UpperRoot.DecRef()
+ }
+ for _, lowerRoot := range fs.opts.LowerRoots {
+ lowerRoot.DecRef()
+ }
+}
+
+func (fs *filesystem) statFS(ctx context.Context) (linux.Statfs, error) {
+ // Always statfs the root of the topmost layer. Compare Linux's
+ // fs/overlayfs/super.c:ovl_statfs().
+ var rootVD vfs.VirtualDentry
+ if fs.opts.UpperRoot.Ok() {
+ rootVD = fs.opts.UpperRoot
+ } else {
+ rootVD = fs.opts.LowerRoots[0]
+ }
+ fsstat, err := fs.vfsfs.VirtualFilesystem().StatFSAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: rootVD,
+ Start: rootVD,
+ })
+ if err != nil {
+ return linux.Statfs{}, err
+ }
+ fsstat.Type = linux.OVERLAYFS_SUPER_MAGIC
+ return fsstat, nil
+}
+
+func (fs *filesystem) newDirIno() uint64 {
+ return atomic.AddUint64(&fs.lastDirIno, 1)
+}
+
+// dentry implements vfs.DentryImpl.
+type dentry struct {
+ vfsd vfs.Dentry
+
+ refs int64
+
+ // fs is the owning filesystem. fs is immutable.
+ fs *filesystem
+
+ // mode, uid, and gid are the file mode, owner, and group of the file in
+ // the topmost layer (and therefore the overlay file as well), and are used
+ // for permission checks on this dentry. These fields are protected by
+ // copyMu and accessed using atomic memory operations.
+ mode uint32
+ uid uint32
+ gid uint32
+
+ // copiedUp is 1 if this dentry has been copied-up (i.e. upperVD.Ok()) and
+ // 0 otherwise. copiedUp is accessed using atomic memory operations.
+ copiedUp uint32
+
+ // parent is the dentry corresponding to this dentry's parent directory.
+ // name is this dentry's name in parent. If this dentry is a filesystem
+ // root, parent is nil and name is the empty string. parent and name are
+ // protected by fs.renameMu.
+ parent *dentry
+ name string
+
+ // If this dentry represents a directory, children maps the names of
+ // children for which dentries have been instantiated to those dentries,
+ // and dirents (if not nil) is a cache of dirents as returned by
+ // directoryFDs representing this directory. children is protected by
+ // dirMu.
+ dirMu sync.Mutex
+ children map[string]*dentry
+ dirents []vfs.Dirent
+
+ // upperVD and lowerVDs are the files from the overlay filesystem's layers
+ // that comprise the file on the overlay filesystem.
+ //
+ // If !upperVD.Ok(), it can transition to a valid vfs.VirtualDentry (i.e.
+ // be copied up) with copyMu locked for writing; otherwise, it is
+ // immutable. lowerVDs is always immutable.
+ copyMu sync.RWMutex
+ upperVD vfs.VirtualDentry
+ lowerVDs []vfs.VirtualDentry
+
+ // inlineLowerVDs backs lowerVDs in the common case where len(lowerVDs) <=
+ // len(inlineLowerVDs).
+ inlineLowerVDs [1]vfs.VirtualDentry
+
+ // devMajor, devMinor, and ino are the device major/minor and inode numbers
+ // used by this dentry. These fields are protected by copyMu and accessed
+ // using atomic memory operations.
+ devMajor uint32
+ devMinor uint32
+ ino uint64
+
+ locks lock.FileLocks
+}
+
+// newDentry creates a new dentry. The dentry initially has no references; it
+// is the caller's responsibility to set the dentry's reference count and/or
+// call dentry.destroy() as appropriate. The dentry is initially invalid in
+// that it contains no layers; the caller is responsible for setting them.
+func (fs *filesystem) newDentry() *dentry {
+ d := &dentry{
+ fs: fs,
+ }
+ d.lowerVDs = d.inlineLowerVDs[:0]
+ d.vfsd.Init(d)
+ return d
+}
+
+// IncRef implements vfs.DentryImpl.IncRef.
+func (d *dentry) IncRef() {
+ // d.refs may be 0 if d.fs.renameMu is locked, which serializes against
+ // d.checkDropLocked().
+ atomic.AddInt64(&d.refs, 1)
+}
+
+// TryIncRef implements vfs.DentryImpl.TryIncRef.
+func (d *dentry) TryIncRef() bool {
+ for {
+ refs := atomic.LoadInt64(&d.refs)
+ if refs <= 0 {
+ return false
+ }
+ if atomic.CompareAndSwapInt64(&d.refs, refs, refs+1) {
+ return true
+ }
+ }
+}
+
+// DecRef implements vfs.DentryImpl.DecRef.
+func (d *dentry) DecRef() {
+ if refs := atomic.AddInt64(&d.refs, -1); refs == 0 {
+ d.fs.renameMu.Lock()
+ d.checkDropLocked()
+ d.fs.renameMu.Unlock()
+ } else if refs < 0 {
+ panic("overlay.dentry.DecRef() called without holding a reference")
+ }
+}
+
+// checkDropLocked should be called after d's reference count becomes 0 or it
+// becomes deleted.
+//
+// Preconditions: d.fs.renameMu must be locked for writing.
+func (d *dentry) checkDropLocked() {
+ // Dentries with a positive reference count must be retained. (The only way
+ // to obtain a reference on a dentry with zero references is via path
+ // resolution, which requires renameMu, so if d.refs is zero then it will
+ // remain zero while we hold renameMu for writing.) Dentries with a
+ // negative reference count have already been destroyed.
+ if atomic.LoadInt64(&d.refs) != 0 {
+ return
+ }
+ // Refs is still zero; destroy it.
+ d.destroyLocked()
+ return
+}
+
+// destroyLocked destroys the dentry.
+//
+// Preconditions: d.fs.renameMu must be locked for writing. d.refs == 0.
+func (d *dentry) destroyLocked() {
+ switch atomic.LoadInt64(&d.refs) {
+ case 0:
+ // Mark the dentry destroyed.
+ atomic.StoreInt64(&d.refs, -1)
+ case -1:
+ panic("overlay.dentry.destroyLocked() called on already destroyed dentry")
+ default:
+ panic("overlay.dentry.destroyLocked() called with references on the dentry")
+ }
+
+ if d.upperVD.Ok() {
+ d.upperVD.DecRef()
+ }
+ for _, lowerVD := range d.lowerVDs {
+ lowerVD.DecRef()
+ }
+
+ if d.parent != nil {
+ d.parent.dirMu.Lock()
+ if !d.vfsd.IsDead() {
+ delete(d.parent.children, d.name)
+ }
+ d.parent.dirMu.Unlock()
+ // Drop the reference held by d on its parent without recursively
+ // locking d.fs.renameMu.
+ if refs := atomic.AddInt64(&d.parent.refs, -1); refs == 0 {
+ d.parent.checkDropLocked()
+ } else if refs < 0 {
+ panic("overlay.dentry.DecRef() called without holding a reference")
+ }
+ }
+}
+
+// InotifyWithParent implements vfs.DentryImpl.InotifyWithParent.
+func (d *dentry) InotifyWithParent(events uint32, cookie uint32, et vfs.EventType) {
+ // TODO(gvisor.dev/issue/1479): Implement inotify.
+}
+
+// Watches implements vfs.DentryImpl.Watches.
+func (d *dentry) Watches() *vfs.Watches {
+ // TODO(gvisor.dev/issue/1479): Implement inotify.
+ return nil
+}
+
+// iterLayers invokes yield on each layer comprising d, from top to bottom. If
+// any call to yield returns false, iterLayer stops iteration.
+func (d *dentry) iterLayers(yield func(vd vfs.VirtualDentry, isUpper bool) bool) {
+ if d.isCopiedUp() {
+ if !yield(d.upperVD, true) {
+ return
+ }
+ }
+ for _, lowerVD := range d.lowerVDs {
+ if !yield(lowerVD, false) {
+ return
+ }
+ }
+}
+
+func (d *dentry) topLayerInfo() (vd vfs.VirtualDentry, isUpper bool) {
+ if d.isCopiedUp() {
+ return d.upperVD, true
+ }
+ return d.lowerVDs[0], false
+}
+
+func (d *dentry) topLayer() vfs.VirtualDentry {
+ vd, _ := d.topLayerInfo()
+ return vd
+}
+
+func (d *dentry) checkPermissions(creds *auth.Credentials, ats vfs.AccessTypes) error {
+ return vfs.GenericCheckPermissions(creds, ats, linux.FileMode(atomic.LoadUint32(&d.mode)), auth.KUID(atomic.LoadUint32(&d.uid)), auth.KGID(atomic.LoadUint32(&d.gid)))
+}
+
+// statInternalMask is the set of stat fields that is set by
+// dentry.statInternalTo().
+const statInternalMask = linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID | linux.STATX_INO
+
+// statInternalTo writes fields to stat that are stored in d, and therefore do
+// not requiring invoking StatAt on the overlay's layers.
+func (d *dentry) statInternalTo(ctx context.Context, opts *vfs.StatOptions, stat *linux.Statx) {
+ stat.Mask |= statInternalMask
+ if d.isDir() {
+ // Linux sets nlink to 1 for merged directories
+ // (fs/overlayfs/inode.c:ovl_getattr()); we set it to 2 because this is
+ // correct more often ("." and the directory's entry in its parent),
+ // and some of our tests expect this.
+ stat.Nlink = 2
+ }
+ stat.UID = atomic.LoadUint32(&d.uid)
+ stat.GID = atomic.LoadUint32(&d.gid)
+ stat.Mode = uint16(atomic.LoadUint32(&d.mode))
+ stat.Ino = atomic.LoadUint64(&d.ino)
+ stat.DevMajor = atomic.LoadUint32(&d.devMajor)
+ stat.DevMinor = atomic.LoadUint32(&d.devMinor)
+}
+
+// Preconditions: d.copyMu must be locked for writing.
+func (d *dentry) updateAfterSetStatLocked(opts *vfs.SetStatOptions) {
+ if opts.Stat.Mask&linux.STATX_MODE != 0 {
+ atomic.StoreUint32(&d.mode, (d.mode&linux.S_IFMT)|uint32(opts.Stat.Mode&^linux.S_IFMT))
+ }
+ if opts.Stat.Mask&linux.STATX_UID != 0 {
+ atomic.StoreUint32(&d.uid, opts.Stat.UID)
+ }
+ if opts.Stat.Mask&linux.STATX_GID != 0 {
+ atomic.StoreUint32(&d.gid, opts.Stat.GID)
+ }
+}
+
+// fileDescription is embedded by overlay implementations of
+// vfs.FileDescriptionImpl.
+type fileDescription struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.LockFD
+}
+
+func (fd *fileDescription) filesystem() *filesystem {
+ return fd.vfsfd.Mount().Filesystem().Impl().(*filesystem)
+}
+
+func (fd *fileDescription) dentry() *dentry {
+ return fd.vfsfd.Dentry().Impl().(*dentry)
+}
diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go
index 97b9b18d7..13c48824e 100644
--- a/pkg/sentry/vfs/file_description.go
+++ b/pkg/sentry/vfs/file_description.go
@@ -108,6 +108,10 @@ type FileDescriptionOptions struct {
UseDentryMetadata bool
}
+// FileCreationFlags are the set of flags passed to FileDescription.Init() but
+// omitted from FileDescription.StatusFlags().
+const FileCreationFlags = linux.O_CREAT | linux.O_EXCL | linux.O_NOCTTY | linux.O_TRUNC
+
// Init must be called before first use of fd. If it succeeds, it takes
// references on mnt and d. flags is the initial file description flags, which
// is usually the full set of flags passed to open(2).
@@ -122,8 +126,8 @@ func (fd *FileDescription) Init(impl FileDescriptionImpl, flags uint32, mnt *Mou
fd.refs = 1
// Remove "file creation flags" to mirror the behavior from file.f_flags in
- // fs/open.c:do_dentry_open
- fd.statusFlags = flags &^ (linux.O_CREAT | linux.O_EXCL | linux.O_NOCTTY | linux.O_TRUNC)
+ // fs/open.c:do_dentry_open.
+ fd.statusFlags = flags &^ FileCreationFlags
fd.vd = VirtualDentry{
mount: mnt,
dentry: d,
@@ -471,6 +475,15 @@ type IterDirentsCallback interface {
Handle(dirent Dirent) error
}
+// IterDirentsCallbackFunc implements IterDirentsCallback for a function with
+// the semantics of IterDirentsCallback.Handle.
+type IterDirentsCallbackFunc func(dirent Dirent) error
+
+// Handle implements IterDirentsCallback.Handle.
+func (f IterDirentsCallbackFunc) Handle(dirent Dirent) error {
+ return f(dirent)
+}
+
// 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.
diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go
index f86db0999..c73072c42 100644
--- a/pkg/syserror/syserror.go
+++ b/pkg/syserror/syserror.go
@@ -72,6 +72,7 @@ var (
EPERM = error(syscall.EPERM)
EPIPE = error(syscall.EPIPE)
ERANGE = error(syscall.ERANGE)
+ EREMOTE = error(syscall.EREMOTE)
EROFS = error(syscall.EROFS)
ESPIPE = error(syscall.ESPIPE)
ESRCH = error(syscall.ESRCH)