summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/p9/client_file.go16
-rw-r--r--pkg/p9/file.go60
-rw-r--r--pkg/p9/handlers.go28
-rw-r--r--pkg/p9/messages.go84
-rw-r--r--pkg/p9/p9.go28
-rw-r--r--pkg/p9/version.go8
-rw-r--r--pkg/sentry/fsimpl/gofer/BUILD1
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go280
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go6
-rw-r--r--pkg/sentry/fsimpl/gofer/p9file.go7
-rw-r--r--pkg/sentry/fsimpl/gofer/revalidate.go386
-rw-r--r--pkg/sentry/vfs/file_description.go14
-rw-r--r--pkg/sentry/vfs/opath.go4
-rw-r--r--pkg/sentry/vfs/resolving_path.go27
-rw-r--r--pkg/sentry/vfs/vfs.go82
15 files changed, 787 insertions, 244 deletions
diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go
index 7abc82e1b..28396b0ea 100644
--- a/pkg/p9/client_file.go
+++ b/pkg/p9/client_file.go
@@ -121,6 +121,22 @@ func (c *clientFile) WalkGetAttr(components []string) ([]QID, File, AttrMask, At
return rwalkgetattr.QIDs, c.client.newFile(FID(fid)), rwalkgetattr.Valid, rwalkgetattr.Attr, nil
}
+func (c *clientFile) MultiGetAttr(names []string) ([]FullStat, error) {
+ if atomic.LoadUint32(&c.closed) != 0 {
+ return nil, unix.EBADF
+ }
+
+ if !versionSupportsTmultiGetAttr(c.client.version) {
+ return DefaultMultiGetAttr(c, names)
+ }
+
+ rmultigetattr := Rmultigetattr{}
+ if err := c.client.sendRecv(&Tmultigetattr{FID: c.fid, Names: names}, &rmultigetattr); err != nil {
+ return nil, err
+ }
+ return rmultigetattr.Stats, nil
+}
+
// StatFS implements File.StatFS.
func (c *clientFile) StatFS() (FSStat, error) {
if atomic.LoadUint32(&c.closed) != 0 {
diff --git a/pkg/p9/file.go b/pkg/p9/file.go
index c59c6a65b..97e0231d6 100644
--- a/pkg/p9/file.go
+++ b/pkg/p9/file.go
@@ -15,6 +15,8 @@
package p9
import (
+ "errors"
+
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/fd"
)
@@ -72,6 +74,15 @@ type File interface {
// On the server, WalkGetAttr has a read concurrency guarantee.
WalkGetAttr([]string) ([]QID, File, AttrMask, Attr, error)
+ // MultiGetAttr batches up multiple calls to GetAttr(). names is a list of
+ // path components similar to Walk(). If the first component name is empty,
+ // the current file is stat'd and included in the results. If the walk reaches
+ // a file that doesn't exist or not a directory, MultiGetAttr returns the
+ // partial result with no error.
+ //
+ // On the server, MultiGetAttr has a read concurrency guarantee.
+ MultiGetAttr(names []string) ([]FullStat, error)
+
// StatFS returns information about the file system associated with
// this file.
//
@@ -306,6 +317,53 @@ func (DisallowClientCalls) SetAttrClose(SetAttrMask, SetAttr) error {
type DisallowServerCalls struct{}
// Renamed implements File.Renamed.
-func (*clientFile) Renamed(File, string) {
+func (*DisallowServerCalls) Renamed(File, string) {
panic("Renamed should not be called on the client")
}
+
+// DefaultMultiGetAttr implements File.MultiGetAttr() on top of File.
+func DefaultMultiGetAttr(start File, names []string) ([]FullStat, error) {
+ stats := make([]FullStat, 0, len(names))
+ parent := start
+ mask := AttrMaskAll()
+ for i, name := range names {
+ if len(name) == 0 && i == 0 {
+ qid, valid, attr, err := parent.GetAttr(mask)
+ if err != nil {
+ return nil, err
+ }
+ stats = append(stats, FullStat{
+ QID: qid,
+ Valid: valid,
+ Attr: attr,
+ })
+ continue
+ }
+ qids, child, valid, attr, err := parent.WalkGetAttr([]string{name})
+ if parent != start {
+ _ = parent.Close()
+ }
+ if err != nil {
+ if errors.Is(err, unix.ENOENT) {
+ return stats, nil
+ }
+ return nil, err
+ }
+ stats = append(stats, FullStat{
+ QID: qids[0],
+ Valid: valid,
+ Attr: attr,
+ })
+ if attr.Mode.FileType() != ModeDirectory {
+ // Doesn't need to continue if entry is not a dir. Including symlinks
+ // that cannot be followed.
+ _ = child.Close()
+ break
+ }
+ parent = child
+ }
+ if parent != start {
+ _ = parent.Close()
+ }
+ return stats, nil
+}
diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go
index 58312d0cc..758e11b13 100644
--- a/pkg/p9/handlers.go
+++ b/pkg/p9/handlers.go
@@ -1421,3 +1421,31 @@ func (t *Tchannel) handle(cs *connState) message {
}
return rchannel
}
+
+// handle implements handler.handle.
+func (t *Tmultigetattr) handle(cs *connState) message {
+ for i, name := range t.Names {
+ if len(name) == 0 && i == 0 {
+ // Empty name is allowed on the first entry to indicate that the current
+ // FID needs to be included in the result.
+ continue
+ }
+ if err := checkSafeName(name); err != nil {
+ return newErr(err)
+ }
+ }
+ ref, ok := cs.LookupFID(t.FID)
+ if !ok {
+ return newErr(unix.EBADF)
+ }
+ defer ref.DecRef()
+
+ var stats []FullStat
+ if err := ref.safelyRead(func() (err error) {
+ stats, err = ref.file.MultiGetAttr(t.Names)
+ return err
+ }); err != nil {
+ return newErr(err)
+ }
+ return &Rmultigetattr{Stats: stats}
+}
diff --git a/pkg/p9/messages.go b/pkg/p9/messages.go
index cf13cbb69..2ff4694c0 100644
--- a/pkg/p9/messages.go
+++ b/pkg/p9/messages.go
@@ -254,8 +254,8 @@ func (r *Rwalk) decode(b *buffer) {
// encode implements encoder.encode.
func (r *Rwalk) encode(b *buffer) {
b.Write16(uint16(len(r.QIDs)))
- for _, q := range r.QIDs {
- q.encode(b)
+ for i := range r.QIDs {
+ r.QIDs[i].encode(b)
}
}
@@ -2243,8 +2243,8 @@ func (r *Rwalkgetattr) encode(b *buffer) {
r.Valid.encode(b)
r.Attr.encode(b)
b.Write16(uint16(len(r.QIDs)))
- for _, q := range r.QIDs {
- q.encode(b)
+ for i := range r.QIDs {
+ r.QIDs[i].encode(b)
}
}
@@ -2552,6 +2552,80 @@ func (r *Rchannel) String() string {
return fmt.Sprintf("Rchannel{Offset: %d, Length: %d}", r.Offset, r.Length)
}
+// Tmultigetattr is a multi-getattr request.
+type Tmultigetattr struct {
+ // FID is the FID to be walked.
+ FID FID
+
+ // Names are the set of names to be walked.
+ Names []string
+}
+
+// decode implements encoder.decode.
+func (t *Tmultigetattr) decode(b *buffer) {
+ t.FID = b.ReadFID()
+ n := b.Read16()
+ t.Names = t.Names[:0]
+ for i := 0; i < int(n); i++ {
+ t.Names = append(t.Names, b.ReadString())
+ }
+}
+
+// encode implements encoder.encode.
+func (t *Tmultigetattr) encode(b *buffer) {
+ b.WriteFID(t.FID)
+ b.Write16(uint16(len(t.Names)))
+ for _, name := range t.Names {
+ b.WriteString(name)
+ }
+}
+
+// Type implements message.Type.
+func (*Tmultigetattr) Type() MsgType {
+ return MsgTmultigetattr
+}
+
+// String implements fmt.Stringer.
+func (t *Tmultigetattr) String() string {
+ return fmt.Sprintf("Tmultigetattr{FID: %d, Names: %v}", t.FID, t.Names)
+}
+
+// Rmultigetattr is a multi-getattr response.
+type Rmultigetattr struct {
+ // Stats are the set of FullStat returned for each of the names in the
+ // request.
+ Stats []FullStat
+}
+
+// decode implements encoder.decode.
+func (r *Rmultigetattr) decode(b *buffer) {
+ n := b.Read16()
+ r.Stats = r.Stats[:0]
+ for i := 0; i < int(n); i++ {
+ var fs FullStat
+ fs.decode(b)
+ r.Stats = append(r.Stats, fs)
+ }
+}
+
+// encode implements encoder.encode.
+func (r *Rmultigetattr) encode(b *buffer) {
+ b.Write16(uint16(len(r.Stats)))
+ for i := range r.Stats {
+ r.Stats[i].encode(b)
+ }
+}
+
+// Type implements message.Type.
+func (*Rmultigetattr) Type() MsgType {
+ return MsgRmultigetattr
+}
+
+// String implements fmt.Stringer.
+func (r *Rmultigetattr) String() string {
+ return fmt.Sprintf("Rmultigetattr{Stats: %v}", r.Stats)
+}
+
const maxCacheSize = 3
// msgFactory is used to reduce allocations by caching messages for reuse.
@@ -2717,6 +2791,8 @@ func init() {
msgRegistry.register(MsgRallocate, func() message { return &Rallocate{} })
msgRegistry.register(MsgTsetattrclunk, func() message { return &Tsetattrclunk{} })
msgRegistry.register(MsgRsetattrclunk, func() message { return &Rsetattrclunk{} })
+ msgRegistry.register(MsgTmultigetattr, func() message { return &Tmultigetattr{} })
+ msgRegistry.register(MsgRmultigetattr, func() message { return &Rmultigetattr{} })
msgRegistry.register(MsgTchannel, func() message { return &Tchannel{} })
msgRegistry.register(MsgRchannel, func() message { return &Rchannel{} })
}
diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go
index 648cf4b49..3d452a0bd 100644
--- a/pkg/p9/p9.go
+++ b/pkg/p9/p9.go
@@ -402,6 +402,8 @@ const (
MsgRallocate MsgType = 139
MsgTsetattrclunk MsgType = 140
MsgRsetattrclunk MsgType = 141
+ MsgTmultigetattr MsgType = 142
+ MsgRmultigetattr MsgType = 143
MsgTchannel MsgType = 250
MsgRchannel MsgType = 251
)
@@ -1178,3 +1180,29 @@ func (a *AllocateMode) encode(b *buffer) {
}
b.Write32(mask)
}
+
+// FullStat is used in the result of a MultiGetAttr call.
+type FullStat struct {
+ QID QID
+ Valid AttrMask
+ Attr Attr
+}
+
+// String implements fmt.Stringer.
+func (f *FullStat) String() string {
+ return fmt.Sprintf("FullStat{QID: %v, Valid: %v, Attr: %v}", f.QID, f.Valid, f.Attr)
+}
+
+// decode implements encoder.decode.
+func (f *FullStat) decode(b *buffer) {
+ f.QID.decode(b)
+ f.Valid.decode(b)
+ f.Attr.decode(b)
+}
+
+// encode implements encoder.encode.
+func (f *FullStat) encode(b *buffer) {
+ f.QID.encode(b)
+ f.Valid.encode(b)
+ f.Attr.encode(b)
+}
diff --git a/pkg/p9/version.go b/pkg/p9/version.go
index 8d7168ef5..950236162 100644
--- a/pkg/p9/version.go
+++ b/pkg/p9/version.go
@@ -26,7 +26,7 @@ const (
//
// Clients are expected to start requesting this version number and
// to continuously decrement it until a Tversion request succeeds.
- highestSupportedVersion uint32 = 12
+ highestSupportedVersion uint32 = 13
// lowestSupportedVersion is the lowest supported version X in a
// version string of the format 9P2000.L.Google.X.
@@ -179,3 +179,9 @@ func versionSupportsListRemoveXattr(v uint32) bool {
func versionSupportsTsetattrclunk(v uint32) bool {
return v >= 12
}
+
+// versionSupportsTmultiGetAttr returns true if version v supports
+// the TmultiGetAttr message.
+func versionSupportsTmultiGetAttr(v uint32) bool {
+ return v >= 13
+}
diff --git a/pkg/sentry/fsimpl/gofer/BUILD b/pkg/sentry/fsimpl/gofer/BUILD
index 6d5258a9b..52879f871 100644
--- a/pkg/sentry/fsimpl/gofer/BUILD
+++ b/pkg/sentry/fsimpl/gofer/BUILD
@@ -38,6 +38,7 @@ go_library(
"host_named_pipe.go",
"p9file.go",
"regular_file.go",
+ "revalidate.go",
"save_restore.go",
"socket.go",
"special_file.go",
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 4b5621043..40c9243f0 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -169,156 +169,89 @@ func (fs *filesystem) renameMuUnlockAndCheckCaching(ctx context.Context, ds **[]
// * fs.renameMu must be locked.
// * d.dirMu must be locked.
// * !rp.Done().
-// * If !d.cachedMetadataAuthoritative(), then d's cached metadata must be up
-// to date.
+// * If !d.cachedMetadataAuthoritative(), then d and all children that are
+// part of rp must have been revalidated.
//
// Postconditions: The returned dentry's cached metadata is up to date.
-func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, mayFollowSymlinks bool, ds **[]*dentry) (*dentry, error) {
+func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, mayFollowSymlinks bool, ds **[]*dentry) (*dentry, bool, error) {
if !d.isDir() {
- return nil, syserror.ENOTDIR
+ return nil, false, syserror.ENOTDIR
}
if err := d.checkPermissions(rp.Credentials(), vfs.MayExec); err != nil {
- return nil, err
+ return nil, false, err
}
+ followedSymlink := false
afterSymlink:
name := rp.Component()
if name == "." {
rp.Advance()
- return d, nil
+ return d, followedSymlink, nil
}
if name == ".." {
if isRoot, err := rp.CheckRoot(ctx, &d.vfsd); err != nil {
- return nil, err
+ return nil, false, err
} else if isRoot || d.parent == nil {
rp.Advance()
- return d, nil
- }
- // We must assume that d.parent is correct, because if d has been moved
- // elsewhere in the remote filesystem so that its parent has changed,
- // we have no way of determining its new parent's location in the
- // filesystem.
- //
- // Call rp.CheckMount() before updating d.parent's metadata, since if
- // we traverse to another mount then d.parent's metadata is irrelevant.
- if err := rp.CheckMount(ctx, &d.parent.vfsd); err != nil {
- return nil, err
+ return d, followedSymlink, nil
}
- if d != d.parent && !d.cachedMetadataAuthoritative() {
- if err := d.parent.updateFromGetattr(ctx); err != nil {
- return nil, err
- }
+ if err := rp.CheckMount(ctx, &d.parent.vfsd); err != nil {
+ return nil, false, err
}
rp.Advance()
- return d.parent, nil
+ return d.parent, followedSymlink, nil
}
- child, err := fs.getChildLocked(ctx, rp.VirtualFilesystem(), d, name, ds)
+ child, err := fs.getChildLocked(ctx, d, name, ds)
if err != nil {
- return nil, err
- }
- if child == nil {
- return nil, syserror.ENOENT
+ return nil, false, err
}
if err := rp.CheckMount(ctx, &child.vfsd); err != nil {
- return nil, err
+ return nil, false, err
}
if child.isSymlink() && mayFollowSymlinks && rp.ShouldFollowSymlink() {
target, err := child.readlink(ctx, rp.Mount())
if err != nil {
- return nil, err
+ return nil, false, err
}
if err := rp.HandleSymlink(target); err != nil {
- return nil, err
+ return nil, false, err
}
+ followedSymlink = true
goto afterSymlink // don't check the current directory again
}
rp.Advance()
- return child, nil
+ return child, followedSymlink, nil
}
// getChildLocked returns a dentry representing the child of parent with the
-// given name. If no such child exists, getChildLocked returns (nil, nil).
+// given name. Returns ENOENT if the child doesn't exist.
//
// Preconditions:
// * fs.renameMu must be locked.
// * parent.dirMu must be locked.
// * parent.isDir().
// * name is not "." or "..".
-//
-// Postconditions: If getChildLocked returns a non-nil dentry, its cached
-// metadata is up to date.
-func (fs *filesystem) getChildLocked(ctx context.Context, vfsObj *vfs.VirtualFilesystem, parent *dentry, name string, ds **[]*dentry) (*dentry, error) {
+// * dentry at name has been revalidated
+func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name string, ds **[]*dentry) (*dentry, error) {
if len(name) > maxFilenameLen {
return nil, syserror.ENAMETOOLONG
}
- child, ok := parent.children[name]
- if (ok && fs.opts.interop != InteropModeShared) || parent.isSynthetic() {
- // Whether child is nil or not, it is cached information that is
- // assumed to be correct.
+ if child, ok := parent.children[name]; ok || parent.isSynthetic() {
+ if child == nil {
+ return nil, syserror.ENOENT
+ }
return child, nil
}
- // We either don't have cached information or need to verify that it's
- // still correct, either of which requires a remote lookup. Check if this
- // name is valid before performing the lookup.
- return fs.revalidateChildLocked(ctx, vfsObj, parent, name, child, ds)
-}
-// Preconditions: Same as getChildLocked, plus:
-// * !parent.isSynthetic().
-func (fs *filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.VirtualFilesystem, parent *dentry, name string, child *dentry, ds **[]*dentry) (*dentry, error) {
- if child != nil {
- // Need to lock child.metadataMu because we might be updating child
- // metadata. We need to hold the lock *before* getting metadata from the
- // server and release it after updating local metadata.
- child.metadataMu.Lock()
- }
qid, file, attrMask, attr, err := parent.file.walkGetAttrOne(ctx, name)
- if err != nil && err != syserror.ENOENT {
- if child != nil {
- child.metadataMu.Unlock()
+ if err != nil {
+ if err == syserror.ENOENT {
+ parent.cacheNegativeLookupLocked(name)
}
return nil, err
}
- if child != nil {
- if !file.isNil() && qid.Path == child.qidPath {
- // The file at this path hasn't changed. Just update cached metadata.
- file.close(ctx)
- child.updateFromP9AttrsLocked(attrMask, &attr)
- child.metadataMu.Unlock()
- return child, nil
- }
- child.metadataMu.Unlock()
- if file.isNil() && child.isSynthetic() {
- // We have a synthetic file, and no remote file has arisen to
- // replace it.
- return child, nil
- }
- // The file at this path has changed or no longer exists. Mark the
- // dentry invalidated, and re-evaluate its caching status (i.e. if it
- // has 0 references, drop it). Wait to update parent.children until we
- // know what to replace the existing dentry with (i.e. one of the
- // returns below), to avoid a redundant map access.
- vfsObj.InvalidateDentry(ctx, &child.vfsd)
- if child.isSynthetic() {
- // Normally we don't mark invalidated dentries as deleted since
- // they may still exist (but at a different path), and also for
- // consistency with Linux. However, synthetic files are guaranteed
- // to become unreachable if their dentries are invalidated, so
- // treat their invalidation as deletion.
- child.setDeleted()
- parent.syntheticChildren--
- child.decRefNoCaching()
- parent.dirents = nil
- }
- *ds = appendDentry(*ds, child)
- }
- if file.isNil() {
- // No file exists at this path now. Cache the negative lookup if
- // allowed.
- parent.cacheNegativeLookupLocked(name)
- return nil, nil
- }
+
// Create a new dentry representing the file.
- child, err = fs.newDentry(ctx, file, qid, attrMask, &attr)
+ child, err := fs.newDentry(ctx, file, qid, attrMask, &attr)
if err != nil {
file.close(ctx)
delete(parent.children, name)
@@ -344,14 +277,22 @@ func (fs *filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.Vir
// * If !d.cachedMetadataAuthoritative(), then d's cached metadata must be up
// to date.
func (fs *filesystem) walkParentDirLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, ds **[]*dentry) (*dentry, error) {
+ if err := fs.revalidateParentDir(ctx, rp, d, ds); err != nil {
+ return nil, err
+ }
for !rp.Final() {
d.dirMu.Lock()
- next, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
+ next, followedSymlink, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
d.dirMu.Unlock()
if err != nil {
return nil, err
}
d = next
+ if followedSymlink {
+ if err := fs.revalidateParentDir(ctx, rp, d, ds); err != nil {
+ return nil, err
+ }
+ }
}
if !d.isDir() {
return nil, syserror.ENOTDIR
@@ -364,20 +305,22 @@ func (fs *filesystem) walkParentDirLocked(ctx context.Context, rp *vfs.Resolving
// 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)
- if !d.cachedMetadataAuthoritative() {
- // Get updated metadata for rp.Start() as required by fs.stepLocked().
- if err := d.updateFromGetattr(ctx); err != nil {
- return nil, err
- }
+ if err := fs.revalidatePath(ctx, rp, d, ds); err != nil {
+ return nil, err
}
for !rp.Done() {
d.dirMu.Lock()
- next, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
+ next, followedSymlink, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds)
d.dirMu.Unlock()
if err != nil {
return nil, err
}
d = next
+ if followedSymlink {
+ if err := fs.revalidatePath(ctx, rp, d, ds); err != nil {
+ return nil, err
+ }
+ }
}
if rp.MustBeDir() && !d.isDir() {
return nil, syserror.ENOTDIR
@@ -397,13 +340,6 @@ func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir
fs.renameMu.RLock()
defer fs.renameMuRUnlockAndCheckCaching(ctx, &ds)
start := rp.Start().Impl().(*dentry)
- if !start.cachedMetadataAuthoritative() {
- // Get updated metadata for start as required by
- // fs.walkParentDirLocked().
- if err := start.updateFromGetattr(ctx); err != nil {
- return err
- }
- }
parent, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
if err != nil {
return err
@@ -421,11 +357,14 @@ func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir
if parent.isDeleted() {
return syserror.ENOENT
}
+ if err := fs.revalidateOne(ctx, rp.VirtualFilesystem(), parent, name, &ds); err != nil {
+ return err
+ }
parent.dirMu.Lock()
defer parent.dirMu.Unlock()
- child, err := fs.getChildLocked(ctx, rp.VirtualFilesystem(), parent, name, &ds)
+ child, err := fs.getChildLocked(ctx, parent, name, &ds)
switch {
case err != nil && err != syserror.ENOENT:
return err
@@ -489,13 +428,6 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b
fs.renameMu.RLock()
defer fs.renameMuRUnlockAndCheckCaching(ctx, &ds)
start := rp.Start().Impl().(*dentry)
- if !start.cachedMetadataAuthoritative() {
- // Get updated metadata for start as required by
- // fs.walkParentDirLocked().
- if err := start.updateFromGetattr(ctx); err != nil {
- return err
- }
- }
parent, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
if err != nil {
return err
@@ -521,33 +453,32 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b
return syserror.EISDIR
}
}
+
vfsObj := rp.VirtualFilesystem()
+ if err := fs.revalidateOne(ctx, vfsObj, parent, rp.Component(), &ds); err != nil {
+ return err
+ }
+
mntns := vfs.MountNamespaceFromContext(ctx)
defer mntns.DecRef(ctx)
+
parent.dirMu.Lock()
defer parent.dirMu.Unlock()
- child, ok := parent.children[name]
- if ok && child == nil {
- return syserror.ENOENT
- }
-
- sticky := atomic.LoadUint32(&parent.mode)&linux.ModeSticky != 0
- if sticky {
- if !ok {
- // If the sticky bit is set, we need to retrieve the child to determine
- // whether removing it is allowed.
- child, err = fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
- if err != nil {
- return err
- }
- } else if child != nil && !child.cachedMetadataAuthoritative() {
- // Make sure the dentry representing the file at name is up to date
- // before examining its metadata.
- child, err = fs.revalidateChildLocked(ctx, vfsObj, parent, name, child, &ds)
- if err != nil {
- return err
- }
+ // Load child if sticky bit is set because we need to determine whether
+ // deletion is allowed.
+ var child *dentry
+ if atomic.LoadUint32(&parent.mode)&linux.ModeSticky == 0 {
+ var ok bool
+ child, ok = parent.children[name]
+ if ok && child == nil {
+ // Hit a negative cached entry, child doesn't exist.
+ return syserror.ENOENT
+ }
+ } else {
+ child, _, err = fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
+ if err != nil {
+ return err
}
if err := parent.mayDelete(rp.Credentials(), child); err != nil {
return err
@@ -556,11 +487,7 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b
// If a child dentry exists, prepare to delete it. This should fail if it is
// a mount point. We detect mount points by speculatively calling
- // PrepareDeleteDentry, which fails if child is a mount point. However, we
- // may need to revalidate the file in this case to make sure that it has not
- // been deleted or replaced on the remote fs, in which case the mount point
- // will have disappeared. If calling PrepareDeleteDentry fails again on the
- // up-to-date dentry, we can be sure that it is a mount point.
+ // PrepareDeleteDentry, which fails if child is a mount point.
//
// Also note that if child is nil, then it can't be a mount point.
if child != nil {
@@ -575,23 +502,7 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b
child.dirMu.Lock()
defer child.dirMu.Unlock()
if err := vfsObj.PrepareDeleteDentry(mntns, &child.vfsd); err != nil {
- // We can skip revalidation in several cases:
- // - We are not in InteropModeShared
- // - The parent directory is synthetic, in which case the child must also
- // be synthetic
- // - We already updated the child during the sticky bit check above
- if parent.cachedMetadataAuthoritative() || sticky {
- return err
- }
- child, err = fs.revalidateChildLocked(ctx, vfsObj, parent, name, child, &ds)
- if err != nil {
- return err
- }
- if child != nil {
- if err := vfsObj.PrepareDeleteDentry(mntns, &child.vfsd); err != nil {
- return err
- }
- }
+ return err
}
}
flags := uint32(0)
@@ -723,13 +634,6 @@ func (fs *filesystem) GetParentDentryAt(ctx context.Context, rp *vfs.ResolvingPa
fs.renameMu.RLock()
defer fs.renameMuRUnlockAndCheckCaching(ctx, &ds)
start := rp.Start().Impl().(*dentry)
- if !start.cachedMetadataAuthoritative() {
- // Get updated metadata for start as required by
- // fs.walkParentDirLocked().
- if err := start.updateFromGetattr(ctx); err != nil {
- return nil, err
- }
- }
d, err := fs.walkParentDirLocked(ctx, rp, start, &ds)
if err != nil {
return nil, err
@@ -830,7 +734,7 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
// to creating a synthetic one, i.e. one that is kept entirely in memory.
// Check that we're not overriding an existing file with a synthetic one.
- _, err = fs.stepLocked(ctx, rp, parent, true, ds)
+ _, _, err = fs.stepLocked(ctx, rp, parent, true, ds)
switch {
case err == nil:
// Step succeeded, another file exists.
@@ -891,12 +795,6 @@ func (fs *filesystem) OpenAt(ctx context.Context, rp *vfs.ResolvingPath, opts vf
defer unlock()
start := rp.Start().Impl().(*dentry)
- if !start.cachedMetadataAuthoritative() {
- // Get updated metadata for start as required by fs.stepLocked().
- if err := start.updateFromGetattr(ctx); err != nil {
- return nil, err
- }
- }
if rp.Done() {
// Reject attempts to open mount root directory with O_CREAT.
if mayCreate && rp.MustBeDir() {
@@ -905,6 +803,12 @@ func (fs *filesystem) OpenAt(ctx context.Context, rp *vfs.ResolvingPath, opts vf
if mustCreate {
return nil, syserror.EEXIST
}
+ if !start.cachedMetadataAuthoritative() {
+ // Refresh dentry's attributes before opening.
+ if err := start.updateFromGetattr(ctx); err != nil {
+ return nil, err
+ }
+ }
start.IncRef()
defer start.DecRef(ctx)
unlock()
@@ -926,9 +830,12 @@ afterTrailingSymlink:
if mayCreate && rp.MustBeDir() {
return nil, syserror.EISDIR
}
+ if err := fs.revalidateOne(ctx, rp.VirtualFilesystem(), parent, rp.Component(), &ds); 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)
+ child, _, err := fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
if err == syserror.ENOENT && mayCreate {
if parent.isSynthetic() {
parent.dirMu.Unlock()
@@ -1297,18 +1204,23 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
if err := oldParent.checkPermissions(creds, vfs.MayWrite|vfs.MayExec); err != nil {
return err
}
+
vfsObj := rp.VirtualFilesystem()
+ if err := fs.revalidateOne(ctx, vfsObj, newParent, newName, &ds); err != nil {
+ return err
+ }
+ if err := fs.revalidateOne(ctx, vfsObj, oldParent, oldName, &ds); err != nil {
+ return err
+ }
+
// We need a dentry representing the renamed file since, if it's a
// directory, we need to check for write permission on it.
oldParent.dirMu.Lock()
defer oldParent.dirMu.Unlock()
- renamed, err := fs.getChildLocked(ctx, vfsObj, oldParent, oldName, &ds)
+ renamed, err := fs.getChildLocked(ctx, oldParent, oldName, &ds)
if err != nil {
return err
}
- if renamed == nil {
- return syserror.ENOENT
- }
if err := oldParent.mayDelete(creds, renamed); err != nil {
return err
}
@@ -1337,8 +1249,8 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
if newParent.isDeleted() {
return syserror.ENOENT
}
- replaced, err := fs.getChildLocked(ctx, rp.VirtualFilesystem(), newParent, newName, &ds)
- if err != nil {
+ replaced, err := fs.getChildLocked(ctx, newParent, newName, &ds)
+ if err != nil && err != syserror.ENOENT {
return err
}
var replacedVFSD *vfs.Dentry
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index fb42c5f62..21692d2ac 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -32,9 +32,9 @@
// specialFileFD.mu
// specialFileFD.bufMu
//
-// Locking dentry.dirMu in multiple dentries requires that either ancestor
-// dentries are locked before descendant dentries, or that filesystem.renameMu
-// is locked for writing.
+// Locking dentry.dirMu and dentry.metadataMu in multiple dentries requires that
+// either ancestor dentries are locked before descendant dentries, or that
+// filesystem.renameMu is locked for writing.
package gofer
import (
diff --git a/pkg/sentry/fsimpl/gofer/p9file.go b/pkg/sentry/fsimpl/gofer/p9file.go
index 21b4a96fe..b0a429d42 100644
--- a/pkg/sentry/fsimpl/gofer/p9file.go
+++ b/pkg/sentry/fsimpl/gofer/p9file.go
@@ -238,3 +238,10 @@ func (f p9file) connect(ctx context.Context, flags p9.ConnectFlags) (*fd.FD, err
ctx.UninterruptibleSleepFinish(false)
return fdobj, err
}
+
+func (f p9file) multiGetAttr(ctx context.Context, names []string) ([]p9.FullStat, error) {
+ ctx.UninterruptibleSleepStart(false)
+ stats, err := f.file.MultiGetAttr(names)
+ ctx.UninterruptibleSleepFinish(false)
+ return stats, err
+}
diff --git a/pkg/sentry/fsimpl/gofer/revalidate.go b/pkg/sentry/fsimpl/gofer/revalidate.go
new file mode 100644
index 000000000..8f81f0822
--- /dev/null
+++ b/pkg/sentry/fsimpl/gofer/revalidate.go
@@ -0,0 +1,386 @@
+// Copyright 2021 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 gofer
+
+import (
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
+)
+
+type errPartialRevalidation struct{}
+
+// Error implements error.Error.
+func (errPartialRevalidation) Error() string {
+ return "partial revalidation"
+}
+
+type errRevalidationStepDone struct{}
+
+// Error implements error.Error.
+func (errRevalidationStepDone) Error() string {
+ return "stop revalidation"
+}
+
+// revalidatePath checks cached dentries for external modification. File
+// attributes are refreshed and cache is invalidated in case the dentry has been
+// deleted, or a new file/directory created in its place.
+//
+// Revalidation stops at symlinks and mount points. The caller is responsible
+// for revalidating again after symlinks are resolved and after changing to
+// different mounts.
+//
+// Preconditions:
+// * fs.renameMu must be locked.
+func (fs *filesystem) revalidatePath(ctx context.Context, rpOrig *vfs.ResolvingPath, start *dentry, ds **[]*dentry) error {
+ // Revalidation is done even if start is synthetic in case the path is
+ // something like: ../non_synthetic_file.
+ if fs.opts.interop != InteropModeShared {
+ return nil
+ }
+
+ // Copy resolving path to walk the path for revalidation.
+ rp := rpOrig.Copy()
+ err := fs.revalidate(ctx, rp, start, rp.Done, ds)
+ rp.Release(ctx)
+ return err
+}
+
+// revalidateParentDir does the same as revalidatePath, but stops at the parent.
+//
+// Preconditions:
+// * fs.renameMu must be locked.
+func (fs *filesystem) revalidateParentDir(ctx context.Context, rpOrig *vfs.ResolvingPath, start *dentry, ds **[]*dentry) error {
+ // Revalidation is done even if start is synthetic in case the path is
+ // something like: ../non_synthetic_file and parent is non synthetic.
+ if fs.opts.interop != InteropModeShared {
+ return nil
+ }
+
+ // Copy resolving path to walk the path for revalidation.
+ rp := rpOrig.Copy()
+ err := fs.revalidate(ctx, rp, start, rp.Final, ds)
+ rp.Release(ctx)
+ return err
+}
+
+// revalidateOne does the same as revalidatePath, but checks a single dentry.
+//
+// Preconditions:
+// * fs.renameMu must be locked.
+func (fs *filesystem) revalidateOne(ctx context.Context, vfsObj *vfs.VirtualFilesystem, parent *dentry, name string, ds **[]*dentry) error {
+ // Skip revalidation for interop mode different than InteropModeShared or
+ // if the parent is synthetic (child must be synthetic too, but it cannot be
+ // replaced without first replacing the parent).
+ if parent.cachedMetadataAuthoritative() {
+ return nil
+ }
+
+ parent.dirMu.Lock()
+ child, ok := parent.children[name]
+ parent.dirMu.Unlock()
+ if !ok {
+ return nil
+ }
+
+ state := makeRevalidateState(parent)
+ defer state.release()
+
+ state.add(name, child)
+ return fs.revalidateHelper(ctx, vfsObj, state, ds)
+}
+
+// revalidate revalidates path components in rp until done returns true, or
+// until a mount point or symlink is reached. It may send multiple MultiGetAttr
+// calls to the gofer to handle ".." in the path.
+//
+// Preconditions:
+// * fs.renameMu must be locked.
+// * InteropModeShared is in effect.
+func (fs *filesystem) revalidate(ctx context.Context, rp *vfs.ResolvingPath, start *dentry, done func() bool, ds **[]*dentry) error {
+ state := makeRevalidateState(start)
+ defer state.release()
+
+ // Skip synthetic dentries because the start dentry cannot be replaced in case
+ // it has been created in the remote file system.
+ if !start.isSynthetic() {
+ state.add("", start)
+ }
+
+done:
+ for cur := start; !done(); {
+ var err error
+ cur, err = fs.revalidateStep(ctx, rp, cur, state)
+ if err != nil {
+ switch err.(type) {
+ case errPartialRevalidation:
+ if err := fs.revalidateHelper(ctx, rp.VirtualFilesystem(), state, ds); err != nil {
+ return err
+ }
+
+ // Reset state to release any remaining locks and restart from where
+ // stepping stopped.
+ state.reset()
+ state.start = cur
+
+ // Skip synthetic dentries because the start dentry cannot be replaced in
+ // case it has been created in the remote file system.
+ if !cur.isSynthetic() {
+ state.add("", cur)
+ }
+
+ case errRevalidationStepDone:
+ break done
+
+ default:
+ return err
+ }
+ }
+ }
+ return fs.revalidateHelper(ctx, rp.VirtualFilesystem(), state, ds)
+}
+
+// revalidateStep walks one element of the path and updates revalidationState
+// with the dentry if needed. It may also stop the stepping or ask for a
+// partial revalidation. Partial revalidation requires the caller to revalidate
+// the current revalidationState, release all locks, and resume stepping.
+// In case a symlink is hit, revalidation stops and the caller is responsible
+// for calling revalidate again after the symlink is resolved. Revalidation may
+// also stop for other reasons, like hitting a child not in the cache.
+//
+// Returns:
+// * (dentry, nil): step worked, continue stepping.`
+// * (dentry, errPartialRevalidation): revalidation should be done with the
+// state gathered so far. Then continue stepping with the remainder of the
+// path, starting at `dentry`.
+// * (nil, errRevalidationStepDone): revalidation doesn't need to step any
+// further. It hit a symlink, a mount point, or an uncached dentry.
+//
+// Preconditions:
+// * fs.renameMu must be locked.
+// * !rp.Done().
+// * InteropModeShared is in effect (assumes no negative dentries).
+func (fs *filesystem) revalidateStep(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, state *revalidateState) (*dentry, error) {
+ switch name := rp.Component(); name {
+ case ".":
+ // Do nothing.
+
+ case "..":
+ // Partial revalidation is required when ".." is hit because metadata locks
+ // can only be acquired from parent to child to avoid deadlocks.
+ if isRoot, err := rp.CheckRoot(ctx, &d.vfsd); err != nil {
+ return nil, errRevalidationStepDone{}
+ } else if isRoot || d.parent == nil {
+ rp.Advance()
+ return d, errPartialRevalidation{}
+ }
+ // We must assume that d.parent is correct, because if d has been moved
+ // elsewhere in the remote filesystem so that its parent has changed,
+ // we have no way of determining its new parent's location in the
+ // filesystem.
+ //
+ // Call rp.CheckMount() before updating d.parent's metadata, since if
+ // we traverse to another mount then d.parent's metadata is irrelevant.
+ if err := rp.CheckMount(ctx, &d.parent.vfsd); err != nil {
+ return nil, errRevalidationStepDone{}
+ }
+ rp.Advance()
+ return d.parent, errPartialRevalidation{}
+
+ default:
+ d.dirMu.Lock()
+ child, ok := d.children[name]
+ d.dirMu.Unlock()
+ if !ok {
+ // child is not cached, no need to validate any further.
+ return nil, errRevalidationStepDone{}
+ }
+
+ state.add(name, child)
+
+ // Symlink must be resolved before continuing with revalidation.
+ if child.isSymlink() {
+ return nil, errRevalidationStepDone{}
+ }
+
+ d = child
+ }
+
+ rp.Advance()
+ return d, nil
+}
+
+// revalidateHelper calls the gofer to stat all dentries in `state`. It will
+// update or invalidate dentries in the cache based on the result.
+//
+// Preconditions:
+// * fs.renameMu must be locked.
+// * InteropModeShared is in effect.
+func (fs *filesystem) revalidateHelper(ctx context.Context, vfsObj *vfs.VirtualFilesystem, state *revalidateState, ds **[]*dentry) error {
+ if len(state.names) == 0 {
+ return nil
+ }
+ // Lock metadata on all dentries *before* getting attributes for them.
+ state.lockAllMetadata()
+ stats, err := state.start.file.multiGetAttr(ctx, state.names)
+ if err != nil {
+ return err
+ }
+
+ i := -1
+ for d := state.popFront(); d != nil; d = state.popFront() {
+ i++
+ found := i < len(stats)
+ if i == 0 && len(state.names[0]) == 0 {
+ if found && !d.isSynthetic() {
+ // First dentry is where the search is starting, just update attributes
+ // since it cannot be replaced.
+ d.updateFromP9AttrsLocked(stats[i].Valid, &stats[i].Attr)
+ }
+ d.metadataMu.Unlock()
+ continue
+ }
+
+ // Note that synthetic dentries will always fails the comparison check
+ // below.
+ if !found || d.qidPath != stats[i].QID.Path {
+ d.metadataMu.Unlock()
+ if !found && d.isSynthetic() {
+ // We have a synthetic file, and no remote file has arisen to replace
+ // it.
+ return nil
+ }
+ // The file at this path has changed or no longer exists. Mark the
+ // dentry invalidated, and re-evaluate its caching status (i.e. if it
+ // has 0 references, drop it). The dentry will be reloaded next time it's
+ // accessed.
+ vfsObj.InvalidateDentry(ctx, &d.vfsd)
+
+ name := state.names[i]
+ d.parent.dirMu.Lock()
+
+ if d.isSynthetic() {
+ // Normally we don't mark invalidated dentries as deleted since
+ // they may still exist (but at a different path), and also for
+ // consistency with Linux. However, synthetic files are guaranteed
+ // to become unreachable if their dentries are invalidated, so
+ // treat their invalidation as deletion.
+ d.setDeleted()
+ d.decRefNoCaching()
+ *ds = appendDentry(*ds, d)
+
+ d.parent.syntheticChildren--
+ d.parent.dirents = nil
+ }
+
+ // Since the dirMu was released and reacquired, re-check that the
+ // parent's child with this name is still the same. Do not touch it if
+ // it has been replaced with a different one.
+ if child := d.parent.children[name]; child == d {
+ // Invalidate dentry so it gets reloaded next time it's accessed.
+ delete(d.parent.children, name)
+ }
+ d.parent.dirMu.Unlock()
+
+ return nil
+ }
+
+ // The file at this path hasn't changed. Just update cached metadata.
+ d.updateFromP9AttrsLocked(stats[i].Valid, &stats[i].Attr)
+ d.metadataMu.Unlock()
+ }
+
+ return nil
+}
+
+// revalidateStatePool caches revalidateState instances to save array
+// allocations for dentries and names.
+var revalidateStatePool = sync.Pool{
+ New: func() interface{} {
+ return &revalidateState{}
+ },
+}
+
+// revalidateState keeps state related to a revalidation request. It keeps track
+// of {name, dentry} list being revalidated, as well as metadata locks on the
+// dentries. The list must be in ancestry order, in other words `n` must be
+// `n-1` child.
+type revalidateState struct {
+ // start is the dentry where to start the attributes search.
+ start *dentry
+
+ // List of names of entries to refresh attributes. Names length must be the
+ // same as detries length. They are kept in separate slices because names is
+ // used to call File.MultiGetAttr().
+ names []string
+
+ // dentries is the list of dentries that correspond to the names above.
+ // dentry.metadataMu is acquired as each dentry is added to this list.
+ dentries []*dentry
+
+ // locked indicates if metadata lock has been acquired on dentries.
+ locked bool
+}
+
+func makeRevalidateState(start *dentry) *revalidateState {
+ r := revalidateStatePool.Get().(*revalidateState)
+ r.start = start
+ return r
+}
+
+// release must be called after the caller is done with this object. It releases
+// all metadata locks and resources.
+func (r *revalidateState) release() {
+ r.reset()
+ revalidateStatePool.Put(r)
+}
+
+// Preconditions:
+// * d is a descendant of all dentries in r.dentries.
+func (r *revalidateState) add(name string, d *dentry) {
+ r.names = append(r.names, name)
+ r.dentries = append(r.dentries, d)
+}
+
+func (r *revalidateState) lockAllMetadata() {
+ for _, d := range r.dentries {
+ d.metadataMu.Lock()
+ }
+ r.locked = true
+}
+
+func (r *revalidateState) popFront() *dentry {
+ if len(r.dentries) == 0 {
+ return nil
+ }
+ d := r.dentries[0]
+ r.dentries = r.dentries[1:]
+ return d
+}
+
+// reset releases all metadata locks and resets all fields to allow this
+// instance to be reused.
+func (r *revalidateState) reset() {
+ if r.locked {
+ // Unlock any remaining dentries.
+ for _, d := range r.dentries {
+ d.metadataMu.Unlock()
+ }
+ r.locked = false
+ }
+ r.start = nil
+ r.names = r.names[:0]
+ r.dentries = r.dentries[:0]
+}
diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go
index f612a71b2..176bcc242 100644
--- a/pkg/sentry/vfs/file_description.go
+++ b/pkg/sentry/vfs/file_description.go
@@ -524,7 +524,7 @@ func (fd *FileDescription) Stat(ctx context.Context, opts StatOptions) (linux.St
Start: fd.vd,
})
stat, err := fd.vd.mount.fs.impl.StatAt(ctx, rp, opts)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return stat, err
}
return fd.impl.Stat(ctx, opts)
@@ -539,7 +539,7 @@ func (fd *FileDescription) SetStat(ctx context.Context, opts SetStatOptions) err
Start: fd.vd,
})
err := fd.vd.mount.fs.impl.SetStatAt(ctx, rp, opts)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
return fd.impl.SetStat(ctx, opts)
@@ -555,7 +555,7 @@ func (fd *FileDescription) StatFS(ctx context.Context) (linux.Statfs, error) {
Start: fd.vd,
})
statfs, err := fd.vd.mount.fs.impl.StatFSAt(ctx, rp)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return statfs, err
}
return fd.impl.StatFS(ctx)
@@ -701,7 +701,7 @@ func (fd *FileDescription) ListXattr(ctx context.Context, size uint64) ([]string
Start: fd.vd,
})
names, err := fd.vd.mount.fs.impl.ListXattrAt(ctx, rp, size)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return names, err
}
names, err := fd.impl.ListXattr(ctx, size)
@@ -730,7 +730,7 @@ func (fd *FileDescription) GetXattr(ctx context.Context, opts *GetXattrOptions)
Start: fd.vd,
})
val, err := fd.vd.mount.fs.impl.GetXattrAt(ctx, rp, *opts)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return val, err
}
return fd.impl.GetXattr(ctx, *opts)
@@ -746,7 +746,7 @@ func (fd *FileDescription) SetXattr(ctx context.Context, opts *SetXattrOptions)
Start: fd.vd,
})
err := fd.vd.mount.fs.impl.SetXattrAt(ctx, rp, *opts)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
return fd.impl.SetXattr(ctx, *opts)
@@ -762,7 +762,7 @@ func (fd *FileDescription) RemoveXattr(ctx context.Context, name string) error {
Start: fd.vd,
})
err := fd.vd.mount.fs.impl.RemoveXattrAt(ctx, rp, name)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
return fd.impl.RemoveXattr(ctx, name)
diff --git a/pkg/sentry/vfs/opath.go b/pkg/sentry/vfs/opath.go
index 39fbac987..47848c76b 100644
--- a/pkg/sentry/vfs/opath.go
+++ b/pkg/sentry/vfs/opath.go
@@ -121,7 +121,7 @@ func (fd *opathFD) Stat(ctx context.Context, opts StatOptions) (linux.Statx, err
Start: fd.vfsfd.vd,
})
stat, err := fd.vfsfd.vd.mount.fs.impl.StatAt(ctx, rp, opts)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return stat, err
}
@@ -134,6 +134,6 @@ func (fd *opathFD) StatFS(ctx context.Context) (linux.Statfs, error) {
Start: fd.vfsfd.vd,
})
statfs, err := fd.vfsfd.vd.mount.fs.impl.StatFSAt(ctx, rp)
- vfsObj.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return statfs, err
}
diff --git a/pkg/sentry/vfs/resolving_path.go b/pkg/sentry/vfs/resolving_path.go
index e4fd55012..634c8b097 100644
--- a/pkg/sentry/vfs/resolving_path.go
+++ b/pkg/sentry/vfs/resolving_path.go
@@ -120,6 +120,8 @@ var resolvingPathPool = sync.Pool{
},
}
+// getResolvingPath gets a new ResolvingPath from the pool. Caller must call
+// ResolvingPath.Release() when done.
func (vfs *VirtualFilesystem) getResolvingPath(creds *auth.Credentials, pop *PathOperation) *ResolvingPath {
rp := resolvingPathPool.Get().(*ResolvingPath)
rp.vfs = vfs
@@ -142,7 +144,30 @@ func (vfs *VirtualFilesystem) getResolvingPath(creds *auth.Credentials, pop *Pat
return rp
}
-func (vfs *VirtualFilesystem) putResolvingPath(ctx context.Context, rp *ResolvingPath) {
+// Copy creates another ResolvingPath with the same state as the original.
+// Copies are independent, using the copy does not change the original and
+// vice-versa.
+//
+// Caller must call Resease() when done.
+func (rp *ResolvingPath) Copy() *ResolvingPath {
+ copy := resolvingPathPool.Get().(*ResolvingPath)
+ *copy = *rp // All fields all shallow copiable.
+
+ // Take extra reference for the copy if the original had them.
+ if copy.flags&rpflagsHaveStartRef != 0 {
+ copy.start.IncRef()
+ }
+ if copy.flags&rpflagsHaveMountRef != 0 {
+ copy.mount.IncRef()
+ }
+ // Reset error state.
+ copy.nextStart = nil
+ copy.nextMount = nil
+ return copy
+}
+
+// Release decrements references if needed and returns the object to the pool.
+func (rp *ResolvingPath) Release(ctx context.Context) {
rp.root = VirtualDentry{}
rp.decRefStartAndMount(ctx)
rp.mount = nil
diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go
index 00f1847d8..8b392232a 100644
--- a/pkg/sentry/vfs/vfs.go
+++ b/pkg/sentry/vfs/vfs.go
@@ -208,11 +208,11 @@ func (vfs *VirtualFilesystem) AccessAt(ctx context.Context, creds *auth.Credenti
for {
err := rp.mount.fs.impl.AccessAt(ctx, rp, creds, ats)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -230,11 +230,11 @@ func (vfs *VirtualFilesystem) GetDentryAt(ctx context.Context, creds *auth.Crede
dentry: d,
}
rp.mount.IncRef()
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return vd, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return VirtualDentry{}, err
}
}
@@ -252,7 +252,7 @@ func (vfs *VirtualFilesystem) getParentDirAndName(ctx context.Context, creds *au
}
rp.mount.IncRef()
name := rp.Component()
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return parentVD, name, nil
}
if checkInvariants {
@@ -261,7 +261,7 @@ func (vfs *VirtualFilesystem) getParentDirAndName(ctx context.Context, creds *au
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return VirtualDentry{}, "", err
}
}
@@ -292,7 +292,7 @@ func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credential
for {
err := rp.mount.fs.impl.LinkAt(ctx, rp, oldVD)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
oldVD.DecRef(ctx)
return nil
}
@@ -302,7 +302,7 @@ func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credential
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
oldVD.DecRef(ctx)
return err
}
@@ -331,7 +331,7 @@ func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentia
for {
err := rp.mount.fs.impl.MkdirAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if checkInvariants {
@@ -340,7 +340,7 @@ func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentia
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -366,7 +366,7 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia
for {
err := rp.mount.fs.impl.MknodAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if checkInvariants {
@@ -375,7 +375,7 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -444,7 +444,7 @@ func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credential
for {
fd, err := rp.mount.fs.impl.OpenAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
if opts.FileExec {
if fd.Mount().Flags.NoExec {
@@ -468,7 +468,7 @@ func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credential
return fd, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil, err
}
}
@@ -480,11 +480,11 @@ func (vfs *VirtualFilesystem) ReadlinkAt(ctx context.Context, creds *auth.Creden
for {
target, err := rp.mount.fs.impl.ReadlinkAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return target, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return "", err
}
}
@@ -533,7 +533,7 @@ func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credenti
for {
err := rp.mount.fs.impl.RenameAt(ctx, rp, oldParentVD, oldName, renameOpts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
oldParentVD.DecRef(ctx)
return nil
}
@@ -543,7 +543,7 @@ func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credenti
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
oldParentVD.DecRef(ctx)
return err
}
@@ -569,7 +569,7 @@ func (vfs *VirtualFilesystem) RmdirAt(ctx context.Context, creds *auth.Credentia
for {
err := rp.mount.fs.impl.RmdirAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if checkInvariants {
@@ -578,7 +578,7 @@ func (vfs *VirtualFilesystem) RmdirAt(ctx context.Context, creds *auth.Credentia
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -590,11 +590,11 @@ func (vfs *VirtualFilesystem) SetStatAt(ctx context.Context, creds *auth.Credent
for {
err := rp.mount.fs.impl.SetStatAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -606,11 +606,11 @@ func (vfs *VirtualFilesystem) StatAt(ctx context.Context, creds *auth.Credential
for {
stat, err := rp.mount.fs.impl.StatAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return stat, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return linux.Statx{}, err
}
}
@@ -623,11 +623,11 @@ func (vfs *VirtualFilesystem) StatFSAt(ctx context.Context, creds *auth.Credenti
for {
statfs, err := rp.mount.fs.impl.StatFSAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return statfs, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return linux.Statfs{}, err
}
}
@@ -652,7 +652,7 @@ func (vfs *VirtualFilesystem) SymlinkAt(ctx context.Context, creds *auth.Credent
for {
err := rp.mount.fs.impl.SymlinkAt(ctx, rp, target)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if checkInvariants {
@@ -661,7 +661,7 @@ func (vfs *VirtualFilesystem) SymlinkAt(ctx context.Context, creds *auth.Credent
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -686,7 +686,7 @@ func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credenti
for {
err := rp.mount.fs.impl.UnlinkAt(ctx, rp)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if checkInvariants {
@@ -695,7 +695,7 @@ func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credenti
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -707,7 +707,7 @@ func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.C
for {
bep, err := rp.mount.fs.impl.BoundEndpointAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return bep, nil
}
if checkInvariants {
@@ -716,7 +716,7 @@ func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.C
}
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil, err
}
}
@@ -729,7 +729,7 @@ func (vfs *VirtualFilesystem) ListXattrAt(ctx context.Context, creds *auth.Crede
for {
names, err := rp.mount.fs.impl.ListXattrAt(ctx, rp, size)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return names, nil
}
if err == syserror.ENOTSUP {
@@ -737,11 +737,11 @@ func (vfs *VirtualFilesystem) ListXattrAt(ctx context.Context, creds *auth.Crede
// fs/xattr.c:vfs_listxattr() falls back to allowing the security
// subsystem to return security extended attributes, which by
// default don't exist.
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil, err
}
}
@@ -754,11 +754,11 @@ func (vfs *VirtualFilesystem) GetXattrAt(ctx context.Context, creds *auth.Creden
for {
val, err := rp.mount.fs.impl.GetXattrAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return val, nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return "", err
}
}
@@ -771,11 +771,11 @@ func (vfs *VirtualFilesystem) SetXattrAt(ctx context.Context, creds *auth.Creden
for {
err := rp.mount.fs.impl.SetXattrAt(ctx, rp, *opts)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}
@@ -787,11 +787,11 @@ func (vfs *VirtualFilesystem) RemoveXattrAt(ctx context.Context, creds *auth.Cre
for {
err := rp.mount.fs.impl.RemoveXattrAt(ctx, rp, name)
if err == nil {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return nil
}
if !rp.handleError(ctx, err) {
- vfs.putResolvingPath(ctx, rp)
+ rp.Release(ctx)
return err
}
}