diff options
Diffstat (limited to 'pkg')
-rw-r--r-- | pkg/p9/client_file.go | 16 | ||||
-rw-r--r-- | pkg/p9/file.go | 60 | ||||
-rw-r--r-- | pkg/p9/handlers.go | 28 | ||||
-rw-r--r-- | pkg/p9/messages.go | 84 | ||||
-rw-r--r-- | pkg/p9/p9.go | 28 | ||||
-rw-r--r-- | pkg/p9/version.go | 8 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/gofer/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/gofer/filesystem.go | 280 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/gofer/gofer.go | 6 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/gofer/p9file.go | 7 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/gofer/revalidate.go | 386 | ||||
-rw-r--r-- | pkg/sentry/vfs/file_description.go | 14 | ||||
-rw-r--r-- | pkg/sentry/vfs/opath.go | 4 | ||||
-rw-r--r-- | pkg/sentry/vfs/resolving_path.go | 27 | ||||
-rw-r--r-- | pkg/sentry/vfs/vfs.go | 82 |
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 } } |