diff options
-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/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 | ||||
-rw-r--r-- | runsc/fsgofer/fsgofer.go | 57 |
15 files changed, 247 insertions, 840 deletions
diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index 28396b0ea..7abc82e1b 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -121,22 +121,6 @@ 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 97e0231d6..c59c6a65b 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -15,8 +15,6 @@ package p9 import ( - "errors" - "golang.org/x/sys/unix" "gvisor.dev/gvisor/pkg/fd" ) @@ -74,15 +72,6 @@ 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. // @@ -317,53 +306,6 @@ func (DisallowClientCalls) SetAttrClose(SetAttrMask, SetAttr) error { type DisallowServerCalls struct{} // Renamed implements File.Renamed. -func (*DisallowServerCalls) Renamed(File, string) { +func (*clientFile) 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 758e11b13..58312d0cc 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -1421,31 +1421,3 @@ 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 2ff4694c0..cf13cbb69 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 i := range r.QIDs { - r.QIDs[i].encode(b) + for _, q := range r.QIDs { + q.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 i := range r.QIDs { - r.QIDs[i].encode(b) + for _, q := range r.QIDs { + q.encode(b) } } @@ -2552,80 +2552,6 @@ 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. @@ -2791,8 +2717,6 @@ 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 3d452a0bd..648cf4b49 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -402,8 +402,6 @@ const ( MsgRallocate MsgType = 139 MsgTsetattrclunk MsgType = 140 MsgRsetattrclunk MsgType = 141 - MsgTmultigetattr MsgType = 142 - MsgRmultigetattr MsgType = 143 MsgTchannel MsgType = 250 MsgRchannel MsgType = 251 ) @@ -1180,29 +1178,3 @@ 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 950236162..8d7168ef5 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 = 13 + highestSupportedVersion uint32 = 12 // lowestSupportedVersion is the lowest supported version X in a // version string of the format 9P2000.L.Google.X. @@ -179,9 +179,3 @@ 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/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go index 40c9243f0..4b5621043 100644 --- a/pkg/sentry/fsimpl/gofer/filesystem.go +++ b/pkg/sentry/fsimpl/gofer/filesystem.go @@ -169,89 +169,156 @@ 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 and all children that are -// part of rp must have been revalidated. +// * If !d.cachedMetadataAuthoritative(), then d's cached metadata must be up +// to date. // // Postconditions: The returned dentry's cached metadata is up to date. -func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, mayFollowSymlinks bool, ds **[]*dentry) (*dentry, bool, error) { +func (fs *filesystem) stepLocked(ctx context.Context, rp *vfs.ResolvingPath, d *dentry, mayFollowSymlinks bool, ds **[]*dentry) (*dentry, error) { if !d.isDir() { - return nil, false, syserror.ENOTDIR + return nil, syserror.ENOTDIR } if err := d.checkPermissions(rp.Credentials(), vfs.MayExec); err != nil { - return nil, false, err + return nil, err } - followedSymlink := false afterSymlink: name := rp.Component() if name == "." { rp.Advance() - return d, followedSymlink, nil + return d, nil } if name == ".." { if isRoot, err := rp.CheckRoot(ctx, &d.vfsd); err != nil { - return nil, false, err + return nil, err } else if isRoot || d.parent == nil { rp.Advance() - return d, followedSymlink, nil - } + 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, false, err + return nil, err + } + if d != d.parent && !d.cachedMetadataAuthoritative() { + if err := d.parent.updateFromGetattr(ctx); err != nil { + return nil, err + } } rp.Advance() - return d.parent, followedSymlink, nil + return d.parent, nil } - child, err := fs.getChildLocked(ctx, d, name, ds) + child, err := fs.getChildLocked(ctx, rp.VirtualFilesystem(), d, name, ds) if err != nil { - return nil, false, err + return nil, err + } + if child == nil { + return nil, syserror.ENOENT } if err := rp.CheckMount(ctx, &child.vfsd); err != nil { - return nil, false, err + return nil, err } if child.isSymlink() && mayFollowSymlinks && rp.ShouldFollowSymlink() { target, err := child.readlink(ctx, rp.Mount()) if err != nil { - return nil, false, err + return nil, err } if err := rp.HandleSymlink(target); err != nil { - return nil, false, err + return nil, err } - followedSymlink = true goto afterSymlink // don't check the current directory again } rp.Advance() - return child, followedSymlink, nil + return child, nil } // getChildLocked returns a dentry representing the child of parent with the -// given name. Returns ENOENT if the child doesn't exist. +// given name. If no such child exists, getChildLocked returns (nil, nil). // // Preconditions: // * fs.renameMu must be locked. // * parent.dirMu must be locked. // * parent.isDir(). // * name is not "." or "..". -// * dentry at name has been revalidated -func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name string, ds **[]*dentry) (*dentry, error) { +// +// 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) { if len(name) > maxFilenameLen { return nil, syserror.ENAMETOOLONG } - if child, ok := parent.children[name]; ok || parent.isSynthetic() { - if child == nil { - return nil, syserror.ENOENT - } + 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. 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 { - if err == syserror.ENOENT { - parent.cacheNegativeLookupLocked(name) + if err != nil && err != syserror.ENOENT { + if child != nil { + child.metadataMu.Unlock() } 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) @@ -277,22 +344,14 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s // * 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, followedSymlink, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds) + next, 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 @@ -305,22 +364,20 @@ 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 err := fs.revalidatePath(ctx, rp, d, ds); err != nil { - return nil, err + if !d.cachedMetadataAuthoritative() { + // Get updated metadata for rp.Start() as required by fs.stepLocked(). + if err := d.updateFromGetattr(ctx); err != nil { + return nil, err + } } for !rp.Done() { d.dirMu.Lock() - next, followedSymlink, err := fs.stepLocked(ctx, rp, d, true /* mayFollowSymlinks */, ds) + next, 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 @@ -340,6 +397,13 @@ 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 @@ -357,14 +421,11 @@ 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, parent, name, &ds) + child, err := fs.getChildLocked(ctx, rp.VirtualFilesystem(), parent, name, &ds) switch { case err != nil && err != syserror.ENOENT: return err @@ -428,6 +489,13 @@ 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 @@ -453,32 +521,33 @@ 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() - // 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 + 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 + } } if err := parent.mayDelete(rp.Credentials(), child); err != nil { return err @@ -487,7 +556,11 @@ 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. + // 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. // // Also note that if child is nil, then it can't be a mount point. if child != nil { @@ -502,7 +575,23 @@ 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 { - return err + // 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 + } + } } } flags := uint32(0) @@ -634,6 +723,13 @@ 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 @@ -734,7 +830,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. @@ -795,6 +891,12 @@ 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() { @@ -803,12 +905,6 @@ 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() @@ -830,12 +926,9 @@ 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() @@ -1204,23 +1297,18 @@ 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, oldParent, oldName, &ds) + renamed, err := fs.getChildLocked(ctx, vfsObj, oldParent, oldName, &ds) if err != nil { return err } + if renamed == nil { + return syserror.ENOENT + } if err := oldParent.mayDelete(creds, renamed); err != nil { return err } @@ -1249,8 +1337,8 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa if newParent.isDeleted() { return syserror.ENOENT } - replaced, err := fs.getChildLocked(ctx, newParent, newName, &ds) - if err != nil && err != syserror.ENOENT { + replaced, err := fs.getChildLocked(ctx, rp.VirtualFilesystem(), newParent, newName, &ds) + if err != nil { return err } var replacedVFSD *vfs.Dentry diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go index 21692d2ac..fb42c5f62 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 and dentry.metadataMu in multiple dentries requires that -// either ancestor dentries are locked before descendant dentries, or that -// filesystem.renameMu is locked for writing. +// Locking dentry.dirMu 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 b0a429d42..21b4a96fe 100644 --- a/pkg/sentry/fsimpl/gofer/p9file.go +++ b/pkg/sentry/fsimpl/gofer/p9file.go @@ -238,10 +238,3 @@ 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 deleted file mode 100644 index 8f81f0822..000000000 --- a/pkg/sentry/fsimpl/gofer/revalidate.go +++ /dev/null @@ -1,386 +0,0 @@ -// 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 176bcc242..f612a71b2 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) return err } return fd.impl.RemoveXattr(ctx, name) diff --git a/pkg/sentry/vfs/opath.go b/pkg/sentry/vfs/opath.go index 47848c76b..39fbac987 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) 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) - rp.Release(ctx) + vfsObj.putResolvingPath(ctx, rp) return statfs, err } diff --git a/pkg/sentry/vfs/resolving_path.go b/pkg/sentry/vfs/resolving_path.go index 634c8b097..e4fd55012 100644 --- a/pkg/sentry/vfs/resolving_path.go +++ b/pkg/sentry/vfs/resolving_path.go @@ -120,8 +120,6 @@ 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 @@ -144,30 +142,7 @@ func (vfs *VirtualFilesystem) getResolvingPath(creds *auth.Credentials, pop *Pat return rp } -// 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) { +func (vfs *VirtualFilesystem) putResolvingPath(ctx context.Context, rp *ResolvingPath) { rp.root = VirtualDentry{} rp.decRefStartAndMount(ctx) rp.mount = nil diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go index 8b392232a..00f1847d8 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return err } } @@ -230,11 +230,11 @@ func (vfs *VirtualFilesystem) GetDentryAt(ctx context.Context, creds *auth.Crede dentry: d, } rp.mount.IncRef() - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return vd, nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return VirtualDentry{}, err } } @@ -252,7 +252,7 @@ func (vfs *VirtualFilesystem) getParentDirAndName(ctx context.Context, creds *au } rp.mount.IncRef() name := rp.Component() - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return parentVD, name, nil } if checkInvariants { @@ -261,7 +261,7 @@ func (vfs *VirtualFilesystem) getParentDirAndName(ctx context.Context, creds *au } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) oldVD.DecRef(ctx) return nil } @@ -302,7 +302,7 @@ func (vfs *VirtualFilesystem) LinkAt(ctx context.Context, creds *auth.Credential } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if checkInvariants { @@ -340,7 +340,7 @@ func (vfs *VirtualFilesystem) MkdirAt(ctx context.Context, creds *auth.Credentia } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if checkInvariants { @@ -375,7 +375,7 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return target, nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) oldParentVD.DecRef(ctx) return nil } @@ -543,7 +543,7 @@ func (vfs *VirtualFilesystem) RenameAt(ctx context.Context, creds *auth.Credenti } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if checkInvariants { @@ -578,7 +578,7 @@ func (vfs *VirtualFilesystem) RmdirAt(ctx context.Context, creds *auth.Credentia } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return stat, nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return statfs, nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if checkInvariants { @@ -661,7 +661,7 @@ func (vfs *VirtualFilesystem) SymlinkAt(ctx context.Context, creds *auth.Credent } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if checkInvariants { @@ -695,7 +695,7 @@ func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credenti } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return bep, nil } if checkInvariants { @@ -716,7 +716,7 @@ func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.C } } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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. - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil, nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return val, nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) 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 { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return nil } if !rp.handleError(ctx, err) { - rp.Release(ctx) + vfs.putResolvingPath(ctx, rp) return err } } diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index b81ede5ae..e04ddda47 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -21,7 +21,6 @@ package fsgofer import ( - "errors" "fmt" "io" "math" @@ -59,6 +58,9 @@ var verityXattrs = map[string]struct{}{ // join is equivalent to path.Join() but skips path.Clean() which is expensive. func join(parent, child string) string { + if child == "." || child == ".." { + panic(fmt.Sprintf("invalid child path %q", child)) + } return parent + "/" + child } @@ -1224,56 +1226,3 @@ func (l *localFile) checkROMount() error { } return nil } - -func (l *localFile) MultiGetAttr(names []string) ([]p9.FullStat, error) { - stats := make([]p9.FullStat, 0, len(names)) - - if len(names) > 0 && names[0] == "" { - qid, valid, attr, err := l.GetAttr(p9.AttrMask{}) - if err != nil { - return nil, err - } - stats = append(stats, p9.FullStat{ - QID: qid, - Valid: valid, - Attr: attr, - }) - names = names[1:] - } - - parent := l.file.FD() - for _, name := range names { - child, err := unix.Openat(parent, name, openFlags|unix.O_PATH, 0) - if parent != l.file.FD() { - // Parent is no longer needed. - _ = unix.Close(parent) - } - if err != nil { - if errors.Is(err, unix.ENOENT) { - // No pont in continuing any further. - break - } - return nil, err - } - - var stat unix.Stat_t - if err := unix.Fstat(child, &stat); err != nil { - _ = unix.Close(child) - return nil, err - } - valid, attr := l.fillAttr(&stat) - stats = append(stats, p9.FullStat{ - QID: l.attachPoint.makeQID(&stat), - Valid: valid, - Attr: attr, - }) - if (stat.Mode & unix.S_IFMT) != unix.S_IFDIR { - // Doesn't need to continue if entry is not a dir. Including symlinks - // that cannot be followed. - _ = unix.Close(child) - break - } - parent = child - } - return stats, nil -} |