From 492229d0176c1af2ab4ea4cf91bf211e940b5b12 Mon Sep 17 00:00:00 2001 From: Jamie Liu Date: Tue, 4 Feb 2020 11:28:36 -0800 Subject: VFS2 gofer client Updates #1198 Opening host pipes (by spinning in fdpipe) and host sockets is not yet complete, and will be done in a future CL. Major differences from VFS1 gofer client (sentry/fs/gofer), with varying levels of backportability: - "Cache policies" are replaced by InteropMode, which control the behavior of timestamps in addition to caching. Under InteropModeExclusive (analogous to cacheAll) and InteropModeWritethrough (analogous to cacheAllWritethrough), client timestamps are *not* written back to the server (it is not possible in 9P or Linux for clients to set ctime, so writing back client-authoritative timestamps results in incoherence between atime/mtime and ctime). Under InteropModeShared (analogous to cacheRemoteRevalidating), client timestamps are not used at all (remote filesystem clocks are authoritative). cacheNone is translated to InteropModeShared + new option filesystemOptions.specialRegularFiles. - Under InteropModeShared, "unstable attribute" reloading for permission checks, lookup, and revalidation are fused, which is feasible in VFS2 since gofer.filesystem controls path resolution. This results in a ~33% reduction in RPCs for filesystem operations compared to cacheRemoteRevalidating. For example, consider stat("/foo/bar/baz") where "/foo/bar/baz" fails revalidation, resulting in the instantiation of a new dentry: VFS1 RPCs: getattr("/") // fs.MountNamespace.FindLink() => fs.Inode.CheckPermission() => gofer.inodeOperations.check() => gofer.inodeOperations.UnstableAttr() walkgetattr("/", "foo") = fid1 // fs.Dirent.walk() => gofer.session.Revalidate() => gofer.cachePolicy.Revalidate() clunk(fid1) getattr("/foo") // CheckPermission walkgetattr("/foo", "bar") = fid2 // Revalidate clunk(fid2) getattr("/foo/bar") // CheckPermission walkgetattr("/foo/bar", "baz") = fid3 // Revalidate clunk(fid3) walkgetattr("/foo/bar", "baz") = fid4 // fs.Dirent.walk() => gofer.inodeOperations.Lookup getattr("/foo/bar/baz") // linux.stat() => gofer.inodeOperations.UnstableAttr() VFS2 RPCs: getattr("/") // gofer.filesystem.walkExistingLocked() walkgetattr("/", "foo") = fid1 // gofer.filesystem.stepExistingLocked() clunk(fid1) // No getattr: walkgetattr already updated metadata for permission check walkgetattr("/foo", "bar") = fid2 clunk(fid2) walkgetattr("/foo/bar", "baz") = fid3 // No clunk: fid3 used for new gofer.dentry // No getattr: walkgetattr already updated metadata for stat() - gofer.filesystem.unlinkAt() does not require instantiation of a dentry that represents the file to be deleted. Updates #898. - gofer.regularFileFD.OnClose() skips Tflushf for regular files under InteropModeExclusive, as it's nonsensical to request a remote file flush without flushing locally-buffered writes to that remote file first. - Symlink targets are cached when InteropModeShared is not in effect. - p9.QID.Path (which is already required to be unique for each file within a server, and is accordingly already synthesized from device/inode numbers in all known gofers) is used as-is for inode numbers, rather than being mapped along with attr.RDev in the client to yet another synthetic inode number. - Relevant parts of fsutil.CachingInodeOperations are inlined directly into gofer package code. This avoids having to duplicate part of its functionality in fsutil.HostMappable. PiperOrigin-RevId: 293190213 --- pkg/sentry/fsimpl/gofer/time.go | 75 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 75 insertions(+) create mode 100644 pkg/sentry/fsimpl/gofer/time.go (limited to 'pkg/sentry/fsimpl/gofer/time.go') diff --git a/pkg/sentry/fsimpl/gofer/time.go b/pkg/sentry/fsimpl/gofer/time.go new file mode 100644 index 000000000..7598ec6a8 --- /dev/null +++ b/pkg/sentry/fsimpl/gofer/time.go @@ -0,0 +1,75 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gofer + +import ( + "sync/atomic" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time" + "gvisor.dev/gvisor/pkg/sentry/vfs" +) + +func dentryTimestampFromP9(s, ns uint64) int64 { + return int64(s*1e9 + ns) +} + +func dentryTimestampFromStatx(ts linux.StatxTimestamp) int64 { + return ts.Sec*1e9 + int64(ts.Nsec) +} + +func statxTimestampFromDentry(ns int64) linux.StatxTimestamp { + return linux.StatxTimestamp{ + Sec: ns / 1e9, + Nsec: uint32(ns % 1e9), + } +} + +func nowFromContext(ctx context.Context) (int64, bool) { + if clock := ktime.RealtimeClockFromContext(ctx); clock != nil { + return clock.Now().Nanoseconds(), true + } + return 0, false +} + +// Preconditions: fs.interop != InteropModeShared. +func (d *dentry) touchAtime(ctx context.Context, mnt *vfs.Mount) { + if err := mnt.CheckBeginWrite(); err != nil { + return + } + now, ok := nowFromContext(ctx) + if !ok { + mnt.EndWrite() + return + } + d.metadataMu.Lock() + atomic.StoreInt64(&d.atime, now) + d.metadataMu.Unlock() + mnt.EndWrite() +} + +// Preconditions: fs.interop != InteropModeShared. The caller has successfully +// called vfs.Mount.CheckBeginWrite(). +func (d *dentry) touchCMtime(ctx context.Context) { + now, ok := nowFromContext(ctx) + if !ok { + return + } + d.metadataMu.Lock() + atomic.StoreInt64(&d.mtime, now) + atomic.StoreInt64(&d.ctime, now) + d.metadataMu.Unlock() +} -- cgit v1.2.3 From dd3bc499970c22ebbd270030b4564e6b8e4e929e Mon Sep 17 00:00:00 2001 From: Fabricio Voznika Date: Thu, 2 Apr 2020 19:37:41 -0700 Subject: Add NAME_MAX checks and update file times NAME_MAX should be enforced per filesystem implementation because other file systems may not have the same restriction. Gofer filesystem now keeps a reference to the kernel clock to avoid lookup in the Context on file access to update atime. Update access, modification, and status change times in tmpfs. Updates #1197, #1198. PiperOrigin-RevId: 304527148 --- pkg/sentry/fsimpl/gofer/directory.go | 7 +++- pkg/sentry/fsimpl/gofer/filesystem.go | 13 ++++++-- pkg/sentry/fsimpl/gofer/gofer.go | 22 ++++++------- pkg/sentry/fsimpl/gofer/regular_file.go | 7 ++-- pkg/sentry/fsimpl/gofer/special_file.go | 4 +-- pkg/sentry/fsimpl/gofer/symlink.go | 2 +- pkg/sentry/fsimpl/gofer/time.go | 39 +++++++++++----------- pkg/sentry/fsimpl/kernfs/filesystem.go | 9 ++++++ pkg/sentry/fsimpl/tmpfs/directory.go | 2 ++ pkg/sentry/fsimpl/tmpfs/filesystem.go | 25 +++++++++++++-- pkg/sentry/fsimpl/tmpfs/regular_file.go | 4 ++- pkg/sentry/fsimpl/tmpfs/tmpfs.go | 57 ++++++++++++++++++++++++++++++--- 12 files changed, 139 insertions(+), 52 deletions(-) (limited to 'pkg/sentry/fsimpl/gofer/time.go') diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go index 5dbfc6250..49d9f859b 100644 --- a/pkg/sentry/fsimpl/gofer/directory.go +++ b/pkg/sentry/fsimpl/gofer/directory.go @@ -56,14 +56,19 @@ func (fd *directoryFD) IterDirents(ctx context.Context, cb vfs.IterDirentsCallba fd.mu.Lock() defer fd.mu.Unlock() + d := fd.dentry() if fd.dirents == nil { - ds, err := fd.dentry().getDirents(ctx) + ds, err := d.getDirents(ctx) if err != nil { return err } fd.dirents = ds } + if d.fs.opts.interop != InteropModeShared { + d.touchAtime(fd.vfsfd.Mount()) + } + for fd.off < int64(len(fd.dirents)) { if err := cb.Handle(fd.dirents[fd.off]); err != nil { return err diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go index 269624362..305228bda 100644 --- a/pkg/sentry/fsimpl/gofer/filesystem.go +++ b/pkg/sentry/fsimpl/gofer/filesystem.go @@ -356,7 +356,9 @@ func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir if err := create(parent, name); err != nil { return err } - parent.touchCMtime(ctx) + if fs.opts.interop != InteropModeShared { + parent.touchCMtime() + } delete(parent.negativeChildren, name) parent.dirents = nil return nil @@ -454,7 +456,7 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b return err } if fs.opts.interop != InteropModeShared { - parent.touchCMtime(ctx) + parent.touchCMtime() if dir { parent.decLinks() } @@ -802,7 +804,6 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving d.IncRef() // reference held by child on its parent d d.vfsd.InsertChild(&child.vfsd, name) if d.fs.opts.interop != InteropModeShared { - d.touchCMtime(ctx) delete(d.negativeChildren, name) d.dirents = nil } @@ -834,6 +835,9 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving } childVFSFD = &fd.vfsfd } + if d.fs.opts.interop != InteropModeShared { + d.touchCMtime() + } return childVFSFD, nil } @@ -975,6 +979,9 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa oldParent.decLinks() newParent.incLinks() } + oldParent.touchCMtime() + newParent.touchCMtime() + renamed.touchCtime() } vfsObj.CommitRenameReplaceDentry(&renamed.vfsd, &newParent.vfsd, newName, replacedVFSD) return nil diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go index 8e41b6b1c..adee8bb60 100644 --- a/pkg/sentry/fsimpl/gofer/gofer.go +++ b/pkg/sentry/fsimpl/gofer/gofer.go @@ -44,6 +44,7 @@ import ( "gvisor.dev/gvisor/pkg/p9" "gvisor.dev/gvisor/pkg/sentry/fs/fsutil" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time" "gvisor.dev/gvisor/pkg/sentry/memmap" "gvisor.dev/gvisor/pkg/sentry/pgalloc" "gvisor.dev/gvisor/pkg/sentry/vfs" @@ -72,6 +73,9 @@ type filesystem struct { // client is the client used by this filesystem. client is immutable. client *p9.Client + // clock is a realtime clock used to set timestamps in file operations. + clock ktime.Clock + // uid and gid are the effective KUID and KGID of the filesystem's creator, // and are used as the owner and group for files that don't specify one. // uid and gid are immutable. @@ -376,6 +380,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt uid: creds.EffectiveKUID, gid: creds.EffectiveKGID, client: client, + clock: ktime.RealtimeClockFromContext(ctx), dentries: make(map[*dentry]struct{}), specialFileFDs: make(map[*specialFileFD]struct{}), } @@ -779,10 +784,7 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *lin // data, so there's no cache to truncate either.) return nil } - now, haveNow := nowFromContext(ctx) - if !haveNow { - ctx.Warningf("gofer.dentry.setStat: current time not available") - } + now := d.fs.clock.Now().Nanoseconds() if stat.Mask&linux.STATX_MODE != 0 { atomic.StoreUint32(&d.mode, d.fileType()|uint32(stat.Mode)) } @@ -794,25 +796,19 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *lin } if setLocalAtime { if stat.Atime.Nsec == linux.UTIME_NOW { - if haveNow { - atomic.StoreInt64(&d.atime, now) - } + atomic.StoreInt64(&d.atime, now) } else { atomic.StoreInt64(&d.atime, dentryTimestampFromStatx(stat.Atime)) } } if setLocalMtime { if stat.Mtime.Nsec == linux.UTIME_NOW { - if haveNow { - atomic.StoreInt64(&d.mtime, now) - } + atomic.StoreInt64(&d.mtime, now) } else { atomic.StoreInt64(&d.mtime, dentryTimestampFromStatx(stat.Mtime)) } } - if haveNow { - atomic.StoreInt64(&d.ctime, now) - } + atomic.StoreInt64(&d.ctime, now) if stat.Mask&linux.STATX_SIZE != 0 { d.dataMu.Lock() oldSize := d.size diff --git a/pkg/sentry/fsimpl/gofer/regular_file.go b/pkg/sentry/fsimpl/gofer/regular_file.go index 3593eb1d5..857f7c74e 100644 --- a/pkg/sentry/fsimpl/gofer/regular_file.go +++ b/pkg/sentry/fsimpl/gofer/regular_file.go @@ -104,7 +104,7 @@ func (fd *regularFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs putDentryReadWriter(rw) if d.fs.opts.interop != InteropModeShared { // Compare Linux's mm/filemap.c:do_generic_file_read() => file_accessed(). - d.touchAtime(ctx, fd.vfsfd.Mount()) + d.touchAtime(fd.vfsfd.Mount()) } return n, err } @@ -139,10 +139,7 @@ func (fd *regularFileFD) PWrite(ctx context.Context, src usermem.IOSequence, off // Compare Linux's mm/filemap.c:__generic_file_write_iter() => // file_update_time(). This is d.touchCMtime(), but without locking // d.metadataMu (recursively). - if now, ok := nowFromContext(ctx); ok { - atomic.StoreInt64(&d.mtime, now) - atomic.StoreInt64(&d.ctime, now) - } + d.touchCMtimeLocked() } if fd.vfsfd.StatusFlags()&linux.O_DIRECT != 0 { // Write dirty cached pages that will be touched by the write back to diff --git a/pkg/sentry/fsimpl/gofer/special_file.go b/pkg/sentry/fsimpl/gofer/special_file.go index 274f7346f..507e0e276 100644 --- a/pkg/sentry/fsimpl/gofer/special_file.go +++ b/pkg/sentry/fsimpl/gofer/special_file.go @@ -76,7 +76,7 @@ func (fd *specialFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs // hold here since specialFileFD doesn't client-cache data. Just buffer the // read instead. if d := fd.dentry(); d.fs.opts.interop != InteropModeShared { - d.touchAtime(ctx, fd.vfsfd.Mount()) + d.touchAtime(fd.vfsfd.Mount()) } buf := make([]byte, dst.NumBytes()) n, err := fd.handle.readToBlocksAt(ctx, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf)), uint64(offset)) @@ -117,7 +117,7 @@ func (fd *specialFileFD) PWrite(ctx context.Context, src usermem.IOSequence, off // Do a buffered write. See rationale in PRead. if d := fd.dentry(); d.fs.opts.interop != InteropModeShared { - d.touchCMtime(ctx) + d.touchCMtime() } buf := make([]byte, src.NumBytes()) // Don't do partial writes if we get a partial read from src. diff --git a/pkg/sentry/fsimpl/gofer/symlink.go b/pkg/sentry/fsimpl/gofer/symlink.go index adf43be60..2ec819f86 100644 --- a/pkg/sentry/fsimpl/gofer/symlink.go +++ b/pkg/sentry/fsimpl/gofer/symlink.go @@ -27,7 +27,7 @@ func (d *dentry) isSymlink() bool { // Precondition: d.isSymlink(). func (d *dentry) readlink(ctx context.Context, mnt *vfs.Mount) (string, error) { if d.fs.opts.interop != InteropModeShared { - d.touchAtime(ctx, mnt) + d.touchAtime(mnt) d.dataMu.Lock() if d.haveTarget { target := d.target diff --git a/pkg/sentry/fsimpl/gofer/time.go b/pkg/sentry/fsimpl/gofer/time.go index 7598ec6a8..2608e7e1d 100644 --- a/pkg/sentry/fsimpl/gofer/time.go +++ b/pkg/sentry/fsimpl/gofer/time.go @@ -18,8 +18,6 @@ import ( "sync/atomic" "gvisor.dev/gvisor/pkg/abi/linux" - "gvisor.dev/gvisor/pkg/context" - ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time" "gvisor.dev/gvisor/pkg/sentry/vfs" ) @@ -38,23 +36,12 @@ func statxTimestampFromDentry(ns int64) linux.StatxTimestamp { } } -func nowFromContext(ctx context.Context) (int64, bool) { - if clock := ktime.RealtimeClockFromContext(ctx); clock != nil { - return clock.Now().Nanoseconds(), true - } - return 0, false -} - // Preconditions: fs.interop != InteropModeShared. -func (d *dentry) touchAtime(ctx context.Context, mnt *vfs.Mount) { +func (d *dentry) touchAtime(mnt *vfs.Mount) { if err := mnt.CheckBeginWrite(); err != nil { return } - now, ok := nowFromContext(ctx) - if !ok { - mnt.EndWrite() - return - } + now := d.fs.clock.Now().Nanoseconds() d.metadataMu.Lock() atomic.StoreInt64(&d.atime, now) d.metadataMu.Unlock() @@ -63,13 +50,25 @@ func (d *dentry) touchAtime(ctx context.Context, mnt *vfs.Mount) { // Preconditions: fs.interop != InteropModeShared. The caller has successfully // called vfs.Mount.CheckBeginWrite(). -func (d *dentry) touchCMtime(ctx context.Context) { - now, ok := nowFromContext(ctx) - if !ok { - return - } +func (d *dentry) touchCtime() { + now := d.fs.clock.Now().Nanoseconds() + d.metadataMu.Lock() + atomic.StoreInt64(&d.ctime, now) + d.metadataMu.Unlock() +} + +// Preconditions: fs.interop != InteropModeShared. The caller has successfully +// called vfs.Mount.CheckBeginWrite(). +func (d *dentry) touchCMtime() { + now := d.fs.clock.Now().Nanoseconds() d.metadataMu.Lock() atomic.StoreInt64(&d.mtime, now) atomic.StoreInt64(&d.ctime, now) d.metadataMu.Unlock() } + +func (d *dentry) touchCMtimeLocked() { + now := d.fs.clock.Now().Nanoseconds() + atomic.StoreInt64(&d.mtime, now) + atomic.StoreInt64(&d.ctime, now) +} diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go index a429fa23d..89f5da3d4 100644 --- a/pkg/sentry/fsimpl/kernfs/filesystem.go +++ b/pkg/sentry/fsimpl/kernfs/filesystem.go @@ -63,6 +63,9 @@ afterSymlink: rp.Advance() return nextVFSD, nil } + if len(name) > linux.NAME_MAX { + return nil, syserror.ENAMETOOLONG + } d.dirMu.Lock() nextVFSD, err := rp.ResolveChild(vfsd, name) if err != nil { @@ -191,6 +194,9 @@ func checkCreateLocked(ctx context.Context, rp *vfs.ResolvingPath, parentVFSD *v if pc == "." || pc == ".." { return "", syserror.EEXIST } + if len(pc) > linux.NAME_MAX { + return "", syserror.ENAMETOOLONG + } childVFSD, err := rp.ResolveChild(parentVFSD, pc) if err != nil { return "", err @@ -433,6 +439,9 @@ afterTrailingSymlink: if pc == "." || pc == ".." { return nil, syserror.EISDIR } + if len(pc) > linux.NAME_MAX { + return nil, syserror.ENAMETOOLONG + } // Determine whether or not we need to create a file. childVFSD, err := rp.ResolveChild(parentVFSD, pc) if err != nil { diff --git a/pkg/sentry/fsimpl/tmpfs/directory.go b/pkg/sentry/fsimpl/tmpfs/directory.go index 37c75ab64..45712c9b9 100644 --- a/pkg/sentry/fsimpl/tmpfs/directory.go +++ b/pkg/sentry/fsimpl/tmpfs/directory.go @@ -68,6 +68,8 @@ func (fd *directoryFD) IterDirents(ctx context.Context, cb vfs.IterDirentsCallba fs.mu.Lock() defer fs.mu.Unlock() + fd.inode().touchAtime(fd.vfsfd.Mount()) + if fd.off == 0 { if err := cb.Handle(vfs.Dirent{ Name: ".", diff --git a/pkg/sentry/fsimpl/tmpfs/filesystem.go b/pkg/sentry/fsimpl/tmpfs/filesystem.go index 4cf27bf13..1978af69c 100644 --- a/pkg/sentry/fsimpl/tmpfs/filesystem.go +++ b/pkg/sentry/fsimpl/tmpfs/filesystem.go @@ -46,6 +46,9 @@ func stepLocked(rp *vfs.ResolvingPath, d *dentry) (*dentry, error) { return nil, err } afterSymlink: + if len(rp.Component()) > linux.NAME_MAX { + return nil, syserror.ENAMETOOLONG + } nextVFSD, err := rp.ResolveComponent(&d.vfsd) if err != nil { return nil, err @@ -133,6 +136,9 @@ func (fs *filesystem) doCreateAt(rp *vfs.ResolvingPath, dir bool, create func(pa if name == "." || name == ".." { return syserror.EEXIST } + if len(name) > linux.NAME_MAX { + return syserror.ENAMETOOLONG + } // Call parent.vfsd.Child() instead of stepLocked() or rp.ResolveChild(), // because if the child exists we want to return EEXIST immediately instead // of attempting symlink/mount traversal. @@ -153,7 +159,11 @@ func (fs *filesystem) doCreateAt(rp *vfs.ResolvingPath, dir bool, create func(pa return err } defer mnt.EndWrite() - return create(parent, name) + if err := create(parent, name); err != nil { + return err + } + parent.inode.touchCMtime() + return nil } // AccessAt implements vfs.Filesystem.Impl.AccessAt. @@ -328,7 +338,12 @@ afterTrailingSymlink: child := fs.newDentry(fs.newRegularFile(rp.Credentials(), opts.Mode)) parent.vfsd.InsertChild(&child.vfsd, name) parent.inode.impl.(*directory).childList.PushBack(child) - return child.open(ctx, rp, &opts, true) + fd, err := child.open(ctx, rp, &opts, true) + if err != nil { + return nil, err + } + parent.inode.touchCMtime() + return fd, nil } if err != nil { return nil, err @@ -398,6 +413,7 @@ func (fs *filesystem) ReadlinkAt(ctx context.Context, rp *vfs.ResolvingPath) (st if !ok { return "", syserror.EINVAL } + symlink.inode.touchAtime(rp.Mount()) return symlink.target, nil } @@ -515,6 +531,9 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa oldParent.inode.decLinksLocked() newParent.inode.incLinksLocked() } + oldParent.inode.touchCMtime() + newParent.inode.touchCMtime() + renamed.inode.touchCtime() // TODO(gvisor.dev/issue/1197): Update timestamps and parent directory // sizes. vfsObj.CommitRenameReplaceDentry(renamedVFSD, &newParent.vfsd, newName, replacedVFSD) @@ -565,6 +584,7 @@ func (fs *filesystem) RmdirAt(ctx context.Context, rp *vfs.ResolvingPath) error parent.inode.decLinksLocked() // from child's ".." child.inode.decLinksLocked() vfsObj.CommitDeleteDentry(childVFSD) + parent.inode.touchCMtime() return nil } @@ -654,6 +674,7 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error parent.inode.impl.(*directory).childList.Remove(child) child.inode.decLinksLocked() vfsObj.CommitDeleteDentry(childVFSD) + parent.inode.touchCMtime() return nil } diff --git a/pkg/sentry/fsimpl/tmpfs/regular_file.go b/pkg/sentry/fsimpl/tmpfs/regular_file.go index 26cd65605..57e5e28ec 100644 --- a/pkg/sentry/fsimpl/tmpfs/regular_file.go +++ b/pkg/sentry/fsimpl/tmpfs/regular_file.go @@ -286,7 +286,8 @@ func (fd *regularFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs rw := getRegularFileReadWriter(f, offset) n, err := dst.CopyOutFrom(ctx, rw) putRegularFileReadWriter(rw) - return int64(n), err + fd.inode().touchAtime(fd.vfsfd.Mount()) + return n, err } // Read implements vfs.FileDescriptionImpl.Read. @@ -323,6 +324,7 @@ func (fd *regularFileFD) PWrite(ctx context.Context, src usermem.IOSequence, off f.inode.mu.Lock() rw := getRegularFileReadWriter(f, offset) n, err := src.CopyInTo(ctx, rw) + fd.inode().touchCMtimeLocked() f.inode.mu.Unlock() putRegularFileReadWriter(rw) return n, err diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go index 54da15849..ad47288f8 100644 --- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go +++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go @@ -385,28 +385,41 @@ func (i *inode) setStat(ctx context.Context, creds *auth.Credentials, stat *linu return syserror.EINVAL } } + now := i.clock.Now().Nanoseconds() if mask&linux.STATX_ATIME != 0 { - atomic.StoreInt64(&i.atime, stat.Atime.ToNsecCapped()) + if stat.Atime.Nsec == linux.UTIME_NOW { + atomic.StoreInt64(&i.atime, now) + } else { + atomic.StoreInt64(&i.atime, stat.Atime.ToNsecCapped()) + } needsCtimeBump = true } if mask&linux.STATX_MTIME != 0 { - atomic.StoreInt64(&i.mtime, stat.Mtime.ToNsecCapped()) + if stat.Mtime.Nsec == linux.UTIME_NOW { + atomic.StoreInt64(&i.mtime, now) + } else { + atomic.StoreInt64(&i.mtime, stat.Mtime.ToNsecCapped()) + } needsCtimeBump = true // Ignore the mtime bump, since we just set it ourselves. needsMtimeBump = false } if mask&linux.STATX_CTIME != 0 { - atomic.StoreInt64(&i.ctime, stat.Ctime.ToNsecCapped()) + if stat.Ctime.Nsec == linux.UTIME_NOW { + atomic.StoreInt64(&i.ctime, now) + } else { + atomic.StoreInt64(&i.ctime, stat.Ctime.ToNsecCapped()) + } // Ignore the ctime bump, since we just set it ourselves. needsCtimeBump = false } - now := i.clock.Now().Nanoseconds() if needsMtimeBump { atomic.StoreInt64(&i.mtime, now) } if needsCtimeBump { atomic.StoreInt64(&i.ctime, now) } + i.mu.Unlock() return nil } @@ -484,6 +497,42 @@ func (i *inode) isDir() bool { return linux.FileMode(i.mode).FileType() == linux.S_IFDIR } +func (i *inode) touchAtime(mnt *vfs.Mount) { + if err := mnt.CheckBeginWrite(); err != nil { + return + } + now := i.clock.Now().Nanoseconds() + i.mu.Lock() + atomic.StoreInt64(&i.atime, now) + i.mu.Unlock() + mnt.EndWrite() +} + +// Preconditions: The caller has called vfs.Mount.CheckBeginWrite(). +func (i *inode) touchCtime() { + now := i.clock.Now().Nanoseconds() + i.mu.Lock() + atomic.StoreInt64(&i.ctime, now) + i.mu.Unlock() +} + +// Preconditions: The caller has called vfs.Mount.CheckBeginWrite(). +func (i *inode) touchCMtime() { + now := i.clock.Now().Nanoseconds() + i.mu.Lock() + atomic.StoreInt64(&i.mtime, now) + atomic.StoreInt64(&i.ctime, now) + i.mu.Unlock() +} + +// Preconditions: The caller has called vfs.Mount.CheckBeginWrite() and holds +// inode.mu. +func (i *inode) touchCMtimeLocked() { + now := i.clock.Now().Nanoseconds() + atomic.StoreInt64(&i.mtime, now) + atomic.StoreInt64(&i.ctime, now) +} + // fileDescription is embedded by tmpfs implementations of // vfs.FileDescriptionImpl. type fileDescription struct { -- cgit v1.2.3 From 21b6bc7280f68f43360a008ffd02a4f461ec9fc8 Mon Sep 17 00:00:00 2001 From: Rahat Mahmood Date: Fri, 5 Jun 2020 19:10:28 -0700 Subject: Implement mount(2) and umount2(2) for VFS2. This is mostly syscall plumbing, VFS2 already implements the internals of mounts. In addition to the syscall defintions, the following mount-related mechanisms are updated: - Implement MS_NOATIME for VFS2, but only for tmpfs and goferfs. The other VFS2 filesystems don't implement node-level timestamps yet. - Implement the 'mode', 'uid' and 'gid' mount options for VFS2's tmpfs. - Plumb mount namespace ownership, which is necessary for checking appropriate capabilities during mount(2). Updates #1035 PiperOrigin-RevId: 315035352 --- pkg/sentry/fsimpl/gofer/time.go | 3 + pkg/sentry/fsimpl/tmpfs/tmpfs.go | 41 +++++++- pkg/sentry/kernel/auth/credentials.go | 28 +++++ pkg/sentry/syscalls/linux/vfs2/BUILD | 1 + pkg/sentry/syscalls/linux/vfs2/mount.go | 145 ++++++++++++++++++++++++++ pkg/sentry/syscalls/linux/vfs2/vfs2.go | 4 +- pkg/sentry/vfs/genericfstree/genericfstree.go | 3 +- pkg/sentry/vfs/mount.go | 34 +++--- pkg/sentry/vfs/options.go | 4 + pkg/sentry/vfs/vfs.go | 2 +- runsc/boot/vfs.go | 2 +- 11 files changed, 247 insertions(+), 20 deletions(-) create mode 100644 pkg/sentry/syscalls/linux/vfs2/mount.go (limited to 'pkg/sentry/fsimpl/gofer/time.go') diff --git a/pkg/sentry/fsimpl/gofer/time.go b/pkg/sentry/fsimpl/gofer/time.go index 2608e7e1d..1d5aa82dc 100644 --- a/pkg/sentry/fsimpl/gofer/time.go +++ b/pkg/sentry/fsimpl/gofer/time.go @@ -38,6 +38,9 @@ func statxTimestampFromDentry(ns int64) linux.StatxTimestamp { // Preconditions: fs.interop != InteropModeShared. func (d *dentry) touchAtime(mnt *vfs.Mount) { + if mnt.Flags.NoATime { + return + } if err := mnt.CheckBeginWrite(); err != nil { return } diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go index f0e098702..3777ebdf2 100644 --- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go +++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go @@ -30,6 +30,7 @@ package tmpfs import ( "fmt" "math" + "strconv" "strings" "sync/atomic" @@ -124,14 +125,45 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt } fs.vfsfs.Init(vfsObj, newFSType, &fs) + mopts := vfs.GenericParseMountOptions(opts.Data) + + defaultMode := linux.FileMode(0777) + if modeStr, ok := mopts["mode"]; ok { + mode, err := strconv.ParseUint(modeStr, 8, 32) + if err != nil { + return nil, nil, fmt.Errorf("Mount option \"mode='%v'\" not parsable: %v", modeStr, err) + } + defaultMode = linux.FileMode(mode) + } + + defaultOwnerCreds := creds.Fork() + if uidStr, ok := mopts["uid"]; ok { + uid, err := strconv.ParseInt(uidStr, 10, 32) + if err != nil { + return nil, nil, fmt.Errorf("Mount option \"uid='%v'\" not parsable: %v", uidStr, err) + } + if err := defaultOwnerCreds.SetUID(auth.UID(uid)); err != nil { + return nil, nil, fmt.Errorf("Error using mount option \"uid='%v'\": %v", uidStr, err) + } + } + if gidStr, ok := mopts["gid"]; ok { + gid, err := strconv.ParseInt(gidStr, 10, 32) + if err != nil { + return nil, nil, fmt.Errorf("Mount option \"gid='%v'\" not parsable: %v", gidStr, err) + } + if err := defaultOwnerCreds.SetGID(auth.GID(gid)); err != nil { + return nil, nil, fmt.Errorf("Error using mount option \"gid='%v'\": %v", gidStr, err) + } + } + var root *dentry switch rootFileType { case linux.S_IFREG: - root = fs.newDentry(fs.newRegularFile(creds, 0777)) + root = fs.newDentry(fs.newRegularFile(defaultOwnerCreds, defaultMode)) case linux.S_IFLNK: - root = fs.newDentry(fs.newSymlink(creds, tmpfsOpts.RootSymlinkTarget)) + root = fs.newDentry(fs.newSymlink(defaultOwnerCreds, tmpfsOpts.RootSymlinkTarget)) case linux.S_IFDIR: - root = &fs.newDirectory(creds, 01777).dentry + root = &fs.newDirectory(defaultOwnerCreds, defaultMode).dentry default: fs.vfsfs.DecRef() return nil, nil, fmt.Errorf("invalid tmpfs root file type: %#o", rootFileType) @@ -562,6 +594,9 @@ func (i *inode) isDir() bool { } func (i *inode) touchAtime(mnt *vfs.Mount) { + if mnt.Flags.NoATime { + return + } if err := mnt.CheckBeginWrite(); err != nil { return } diff --git a/pkg/sentry/kernel/auth/credentials.go b/pkg/sentry/kernel/auth/credentials.go index e057d2c6d..6862f2ef5 100644 --- a/pkg/sentry/kernel/auth/credentials.go +++ b/pkg/sentry/kernel/auth/credentials.go @@ -232,3 +232,31 @@ func (c *Credentials) UseGID(gid GID) (KGID, error) { } return NoID, syserror.EPERM } + +// SetUID translates the provided uid to the root user namespace and updates c's +// uids to it. This performs no permissions or capabilities checks, the caller +// is responsible for ensuring the calling context is permitted to modify c. +func (c *Credentials) SetUID(uid UID) error { + kuid := c.UserNamespace.MapToKUID(uid) + if !kuid.Ok() { + return syserror.EINVAL + } + c.RealKUID = kuid + c.EffectiveKUID = kuid + c.SavedKUID = kuid + return nil +} + +// SetGID translates the provided gid to the root user namespace and updates c's +// gids to it. This performs no permissions or capabilities checks, the caller +// is responsible for ensuring the calling context is permitted to modify c. +func (c *Credentials) SetGID(gid GID) error { + kgid := c.UserNamespace.MapToKGID(gid) + if !kgid.Ok() { + return syserror.EINVAL + } + c.RealKGID = kgid + c.EffectiveKGID = kgid + c.SavedKGID = kgid + return nil +} diff --git a/pkg/sentry/syscalls/linux/vfs2/BUILD b/pkg/sentry/syscalls/linux/vfs2/BUILD index 9c8b44f64..c0d005247 100644 --- a/pkg/sentry/syscalls/linux/vfs2/BUILD +++ b/pkg/sentry/syscalls/linux/vfs2/BUILD @@ -16,6 +16,7 @@ go_library( "ioctl.go", "memfd.go", "mmap.go", + "mount.go", "path.go", "pipe.go", "poll.go", diff --git a/pkg/sentry/syscalls/linux/vfs2/mount.go b/pkg/sentry/syscalls/linux/vfs2/mount.go new file mode 100644 index 000000000..adeaa39cc --- /dev/null +++ b/pkg/sentry/syscalls/linux/vfs2/mount.go @@ -0,0 +1,145 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vfs2 + +import ( + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/sentry/arch" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +// Mount implements Linux syscall mount(2). +func Mount(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + sourceAddr := args[0].Pointer() + targetAddr := args[1].Pointer() + typeAddr := args[2].Pointer() + flags := args[3].Uint64() + dataAddr := args[4].Pointer() + + // For null-terminated strings related to mount(2), Linux copies in at most + // a page worth of data. See fs/namespace.c:copy_mount_string(). + fsType, err := t.CopyInString(typeAddr, usermem.PageSize) + if err != nil { + return 0, nil, err + } + source, err := t.CopyInString(sourceAddr, usermem.PageSize) + if err != nil { + return 0, nil, err + } + + targetPath, err := copyInPath(t, targetAddr) + if err != nil { + return 0, nil, err + } + + data := "" + if dataAddr != 0 { + // In Linux, a full page is always copied in regardless of null + // character placement, and the address is passed to each file system. + // Most file systems always treat this data as a string, though, and so + // do all of the ones we implement. + data, err = t.CopyInString(dataAddr, usermem.PageSize) + if err != nil { + return 0, nil, err + } + } + + // Ignore magic value that was required before Linux 2.4. + if flags&linux.MS_MGC_MSK == linux.MS_MGC_VAL { + flags = flags &^ linux.MS_MGC_MSK + } + + // Must have CAP_SYS_ADMIN in the current mount namespace's associated user + // namespace. + creds := t.Credentials() + if !creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, t.MountNamespaceVFS2().Owner) { + return 0, nil, syserror.EPERM + } + + const unsupportedOps = linux.MS_REMOUNT | linux.MS_BIND | + linux.MS_SHARED | linux.MS_PRIVATE | linux.MS_SLAVE | + linux.MS_UNBINDABLE | linux.MS_MOVE + + // Silently allow MS_NOSUID, since we don't implement set-id bits + // anyway. + const unsupportedFlags = linux.MS_NODEV | + linux.MS_NODIRATIME | linux.MS_STRICTATIME + + // Linux just allows passing any flags to mount(2) - it won't fail when + // unknown or unsupported flags are passed. Since we don't implement + // everything, we fail explicitly on flags that are unimplemented. + if flags&(unsupportedOps|unsupportedFlags) != 0 { + return 0, nil, syserror.EINVAL + } + + var opts vfs.MountOptions + if flags&linux.MS_NOATIME == linux.MS_NOATIME { + opts.Flags.NoATime = true + } + if flags&linux.MS_NOEXEC == linux.MS_NOEXEC { + opts.Flags.NoExec = true + } + if flags&linux.MS_RDONLY == linux.MS_RDONLY { + opts.ReadOnly = true + } + opts.GetFilesystemOptions.Data = data + + target, err := getTaskPathOperation(t, linux.AT_FDCWD, targetPath, disallowEmptyPath, nofollowFinalSymlink) + if err != nil { + return 0, nil, err + } + defer target.Release() + + return 0, nil, t.Kernel().VFS().MountAt(t, creds, source, &target.pop, fsType, &opts) +} + +// Umount2 implements Linux syscall umount2(2). +func Umount2(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + addr := args[0].Pointer() + flags := args[1].Int() + + // Must have CAP_SYS_ADMIN in the mount namespace's associated user + // namespace. + // + // Currently, this is always the init task's user namespace. + creds := t.Credentials() + if !creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, t.MountNamespaceVFS2().Owner) { + return 0, nil, syserror.EPERM + } + + const unsupported = linux.MNT_FORCE | linux.MNT_EXPIRE + if flags&unsupported != 0 { + return 0, nil, syserror.EINVAL + } + + path, err := copyInPath(t, addr) + if err != nil { + return 0, nil, err + } + tpop, err := getTaskPathOperation(t, linux.AT_FDCWD, path, disallowEmptyPath, nofollowFinalSymlink) + if err != nil { + return 0, nil, err + } + defer tpop.Release() + + opts := vfs.UmountOptions{ + Flags: uint32(flags), + } + + return 0, nil, t.Kernel().VFS().UmountAt(t, creds, &tpop.pop, &opts) +} diff --git a/pkg/sentry/syscalls/linux/vfs2/vfs2.go b/pkg/sentry/syscalls/linux/vfs2/vfs2.go index ef8358b8a..7b6e7571a 100644 --- a/pkg/sentry/syscalls/linux/vfs2/vfs2.go +++ b/pkg/sentry/syscalls/linux/vfs2/vfs2.go @@ -90,8 +90,8 @@ func Override() { s.Table[138] = syscalls.Supported("fstatfs", Fstatfs) s.Table[161] = syscalls.Supported("chroot", Chroot) s.Table[162] = syscalls.Supported("sync", Sync) - delete(s.Table, 165) // mount - delete(s.Table, 166) // umount2 + s.Table[165] = syscalls.Supported("mount", Mount) + s.Table[166] = syscalls.Supported("umount2", Umount2) delete(s.Table, 187) // readahead s.Table[188] = syscalls.Supported("setxattr", Setxattr) s.Table[189] = syscalls.Supported("lsetxattr", Lsetxattr) diff --git a/pkg/sentry/vfs/genericfstree/genericfstree.go b/pkg/sentry/vfs/genericfstree/genericfstree.go index 286510195..8882fa84a 100644 --- a/pkg/sentry/vfs/genericfstree/genericfstree.go +++ b/pkg/sentry/vfs/genericfstree/genericfstree.go @@ -43,7 +43,7 @@ type Dentry struct { // IsAncestorDentry returns true if d is an ancestor of d2; that is, d is // either d2's parent or an ancestor of d2's parent. func IsAncestorDentry(d, d2 *Dentry) bool { - for { + for d2 != nil { // Stop at root, where d2.parent == nil. if d2.parent == d { return true } @@ -52,6 +52,7 @@ func IsAncestorDentry(d, d2 *Dentry) bool { } d2 = d2.parent } + return false } // ParentOrSelf returns d.parent. If d.parent is nil, ParentOrSelf returns d. diff --git a/pkg/sentry/vfs/mount.go b/pkg/sentry/vfs/mount.go index 3adb7c97d..32f901bd8 100644 --- a/pkg/sentry/vfs/mount.go +++ b/pkg/sentry/vfs/mount.go @@ -55,6 +55,10 @@ type Mount struct { // ID is the immutable mount ID. ID uint64 + // Flags contains settings as specified for mount(2), e.g. MS_NOEXEC, except + // for MS_RDONLY which is tracked in "writers". Immutable. + Flags MountFlags + // key is protected by VirtualFilesystem.mountMu and // VirtualFilesystem.mounts.seq, and may be nil. References are held on // key.parent and key.point if they are not nil. @@ -81,10 +85,6 @@ type Mount struct { // umounted is true. umounted is protected by VirtualFilesystem.mountMu. umounted bool - // flags contains settings as specified for mount(2), e.g. MS_NOEXEC, except - // for MS_RDONLY which is tracked in "writers". - flags MountFlags - // The lower 63 bits of writers is the number of calls to // Mount.CheckBeginWrite() that have not yet been paired with a call to // Mount.EndWrite(). The MSB of writers is set if MS_RDONLY is in effect. @@ -95,10 +95,10 @@ type Mount struct { func newMount(vfs *VirtualFilesystem, fs *Filesystem, root *Dentry, mntns *MountNamespace, opts *MountOptions) *Mount { mnt := &Mount{ ID: atomic.AddUint64(&vfs.lastMountID, 1), + Flags: opts.Flags, vfs: vfs, fs: fs, root: root, - flags: opts.Flags, ns: mntns, refs: 1, } @@ -113,13 +113,12 @@ func (mnt *Mount) Options() MountOptions { mnt.vfs.mountMu.Lock() defer mnt.vfs.mountMu.Unlock() return MountOptions{ - Flags: mnt.flags, + Flags: mnt.Flags, ReadOnly: mnt.readOnly(), } } -// A MountNamespace is a collection of Mounts. -// +// A MountNamespace is a collection of Mounts.// // MountNamespaces are reference-counted. Unless otherwise specified, all // MountNamespace methods require that a reference is held. // @@ -127,6 +126,9 @@ func (mnt *Mount) Options() MountOptions { // // +stateify savable type MountNamespace struct { + // Owner is the usernamespace that owns this mount namespace. + Owner *auth.UserNamespace + // root is the MountNamespace's root mount. root is immutable. root *Mount @@ -163,6 +165,7 @@ func (vfs *VirtualFilesystem) NewMountNamespace(ctx context.Context, creds *auth return nil, err } mntns := &MountNamespace{ + Owner: creds.UserNamespace, refs: 1, mountpoints: make(map[*Dentry]uint32), } @@ -279,6 +282,9 @@ func (vfs *VirtualFilesystem) UmountAt(ctx context.Context, creds *auth.Credenti } // MNT_FORCE is currently unimplemented except for the permission check. + // Force unmounting specifically requires CAP_SYS_ADMIN in the root user + // namespace, and not in the owner user namespace for the target mount. See + // fs/namespace.c:SYSCALL_DEFINE2(umount, ...) if opts.Flags&linux.MNT_FORCE != 0 && creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, creds.UserNamespace.Root()) { return syserror.EPERM } @@ -753,7 +759,10 @@ func (vfs *VirtualFilesystem) GenerateProcMounts(ctx context.Context, taskRootDi if mnt.readOnly() { opts = "ro" } - if mnt.flags.NoExec { + if mnt.Flags.NoATime { + opts = ",noatime" + } + if mnt.Flags.NoExec { opts += ",noexec" } @@ -838,11 +847,12 @@ func (vfs *VirtualFilesystem) GenerateProcMountInfo(ctx context.Context, taskRoo if mnt.readOnly() { opts = "ro" } - if mnt.flags.NoExec { + if mnt.Flags.NoATime { + opts = ",noatime" + } + if mnt.Flags.NoExec { opts += ",noexec" } - // TODO(gvisor.dev/issue/1193): Add "noatime" if MS_NOATIME is - // set. fmt.Fprintf(buf, "%s ", opts) // (7) Optional fields: zero or more fields of the form "tag[:value]". diff --git a/pkg/sentry/vfs/options.go b/pkg/sentry/vfs/options.go index 53d364c5c..f223aeda8 100644 --- a/pkg/sentry/vfs/options.go +++ b/pkg/sentry/vfs/options.go @@ -75,6 +75,10 @@ type MknodOptions struct { type MountFlags struct { // NoExec is equivalent to MS_NOEXEC. NoExec bool + + // NoATime is equivalent to MS_NOATIME and indicates that the + // filesystem should not update access time in-place. + NoATime bool } // MountOptions contains options to VirtualFilesystem.MountAt(). diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go index 52643a7c5..9acca8bc7 100644 --- a/pkg/sentry/vfs/vfs.go +++ b/pkg/sentry/vfs/vfs.go @@ -405,7 +405,7 @@ func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credential vfs.putResolvingPath(rp) if opts.FileExec { - if fd.Mount().flags.NoExec { + if fd.Mount().Flags.NoExec { fd.DecRef() return nil, syserror.EACCES } diff --git a/runsc/boot/vfs.go b/runsc/boot/vfs.go index 6c84f0794..7ed6801b4 100644 --- a/runsc/boot/vfs.go +++ b/runsc/boot/vfs.go @@ -272,7 +272,7 @@ func (c *containerMounter) getMountNameAndOptionsVFS2(conf *Config, m *mountAndF case "ro": opts.ReadOnly = true case "noatime": - // TODO(gvisor.dev/issue/1193): Implement MS_NOATIME. + opts.Flags.NoATime = true case "noexec": opts.Flags.NoExec = true default: -- cgit v1.2.3 From 52b44719d6e14ec299d0d953b4dc07a712b897fa Mon Sep 17 00:00:00 2001 From: Ayush Ranjan Date: Wed, 1 Jul 2020 19:47:51 -0700 Subject: [vfs2][gofer] Update file size to 0 on O_TRUNC Some Open:TruncateXxx syscall tests were failing because the file size was not being updated when the file was opened with O_TRUNC. Fixes Truncate tests in test/syscalls:open_test_runsc_ptrace_vfs2. Updates #2923 PiperOrigin-RevId: 319340127 --- pkg/sentry/fsimpl/gofer/filesystem.go | 35 ++++++++++++++++-- pkg/sentry/fsimpl/gofer/gofer.go | 69 ++++++++++++++++++----------------- pkg/sentry/fsimpl/gofer/time.go | 12 +++--- pkg/sentry/fsimpl/tmpfs/filesystem.go | 2 +- 4 files changed, 76 insertions(+), 42 deletions(-) (limited to 'pkg/sentry/fsimpl/gofer/time.go') diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go index 51082359d..7bcc99b29 100644 --- a/pkg/sentry/fsimpl/gofer/filesystem.go +++ b/pkg/sentry/fsimpl/gofer/filesystem.go @@ -869,11 +869,22 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf if err := d.checkPermissions(rp.Credentials(), ats); err != nil { return nil, err } + + trunc := opts.Flags&linux.O_TRUNC != 0 && d.fileType() == linux.S_IFREG + if trunc { + // Lock metadataMu *while* we open a regular file with O_TRUNC because + // open(2) will change the file size on server. + d.metadataMu.Lock() + defer d.metadataMu.Unlock() + } + + var vfd *vfs.FileDescription + var err error mnt := rp.Mount() switch d.fileType() { case linux.S_IFREG: if !d.fs.opts.regularFilesUseSpecialFileFD { - if err := d.ensureSharedHandle(ctx, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, opts.Flags&linux.O_TRUNC != 0); err != nil { + if err := d.ensureSharedHandle(ctx, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, trunc); err != nil { return nil, err } fd := ®ularFileFD{} @@ -883,7 +894,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf }); err != nil { return nil, err } - return &fd.vfsfd, nil + vfd = &fd.vfsfd } case linux.S_IFDIR: // Can't open directories with O_CREAT. @@ -923,7 +934,25 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf return d.pipe.Open(ctx, mnt, &d.vfsd, opts.Flags, &d.locks) } } - return d.openSpecialFileLocked(ctx, mnt, opts) + + if vfd == nil { + if vfd, err = d.openSpecialFileLocked(ctx, mnt, opts); err != nil { + return nil, err + } + } + + if trunc { + // If no errors occured so far then update file size in memory. This + // step is required even if !d.cachedMetadataAuthoritative() because + // d.mappings has to be updated. + // d.metadataMu has already been acquired if trunc == true. + d.updateFileSizeLocked(0) + + if d.cachedMetadataAuthoritative() { + d.touchCMtimeLocked() + } + } + return vfd, err } func (d *dentry) connectSocketLocked(ctx context.Context, opts *vfs.OpenOptions) (*vfs.FileDescription, error) { diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go index 71c8d3ae1..709b5e496 100644 --- a/pkg/sentry/fsimpl/gofer/gofer.go +++ b/pkg/sentry/fsimpl/gofer/gofer.go @@ -794,9 +794,7 @@ func (d *dentry) updateFromP9Attrs(mask p9.AttrMask, attr *p9.Attr) { atomic.StoreUint32(&d.nlink, uint32(attr.NLink)) } if mask.Size { - d.dataMu.Lock() - atomic.StoreUint64(&d.size, attr.Size) - d.dataMu.Unlock() + d.updateFileSizeLocked(attr.Size) } d.metadataMu.Unlock() } @@ -964,39 +962,44 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *lin } atomic.StoreInt64(&d.ctime, now) if stat.Mask&linux.STATX_SIZE != 0 { + d.updateFileSizeLocked(stat.Size) + } + return nil +} + +// Preconditions: d.metadataMu must be locked. +func (d *dentry) updateFileSizeLocked(newSize uint64) { + d.dataMu.Lock() + oldSize := d.size + d.size = newSize + // d.dataMu must be unlocked to lock d.mapsMu and invalidate mappings + // below. This allows concurrent calls to Read/Translate/etc. These + // functions synchronize with truncation by refusing to use cache + // contents beyond the new d.size. (We are still holding d.metadataMu, + // so we can't race with Write or another truncate.) + d.dataMu.Unlock() + if d.size < oldSize { + oldpgend, _ := usermem.PageRoundUp(oldSize) + newpgend, _ := usermem.PageRoundUp(d.size) + if oldpgend != newpgend { + d.mapsMu.Lock() + d.mappings.Invalidate(memmap.MappableRange{newpgend, oldpgend}, memmap.InvalidateOpts{ + // Compare Linux's mm/truncate.c:truncate_setsize() => + // truncate_pagecache() => + // mm/memory.c:unmap_mapping_range(evencows=1). + InvalidatePrivate: true, + }) + d.mapsMu.Unlock() + } + // We are now guaranteed that there are no translations of + // truncated pages, and can remove them from the cache. Since + // truncated pages have been removed from the remote file, they + // should be dropped without being written back. d.dataMu.Lock() - oldSize := d.size - d.size = stat.Size - // d.dataMu must be unlocked to lock d.mapsMu and invalidate mappings - // below. This allows concurrent calls to Read/Translate/etc. These - // functions synchronize with truncation by refusing to use cache - // contents beyond the new d.size. (We are still holding d.metadataMu, - // so we can't race with Write or another truncate.) + d.cache.Truncate(d.size, d.fs.mfp.MemoryFile()) + d.dirty.KeepClean(memmap.MappableRange{d.size, oldpgend}) d.dataMu.Unlock() - if d.size < oldSize { - oldpgend, _ := usermem.PageRoundUp(oldSize) - newpgend, _ := usermem.PageRoundUp(d.size) - if oldpgend != newpgend { - d.mapsMu.Lock() - d.mappings.Invalidate(memmap.MappableRange{newpgend, oldpgend}, memmap.InvalidateOpts{ - // Compare Linux's mm/truncate.c:truncate_setsize() => - // truncate_pagecache() => - // mm/memory.c:unmap_mapping_range(evencows=1). - InvalidatePrivate: true, - }) - d.mapsMu.Unlock() - } - // We are now guaranteed that there are no translations of - // truncated pages, and can remove them from the cache. Since - // truncated pages have been removed from the remote file, they - // should be dropped without being written back. - d.dataMu.Lock() - d.cache.Truncate(d.size, d.fs.mfp.MemoryFile()) - d.dirty.KeepClean(memmap.MappableRange{d.size, oldpgend}) - d.dataMu.Unlock() - } } - return nil } func (d *dentry) checkPermissions(creds *auth.Credentials, ats vfs.AccessTypes) error { diff --git a/pkg/sentry/fsimpl/gofer/time.go b/pkg/sentry/fsimpl/gofer/time.go index 1d5aa82dc..0eef4e16e 100644 --- a/pkg/sentry/fsimpl/gofer/time.go +++ b/pkg/sentry/fsimpl/gofer/time.go @@ -36,7 +36,7 @@ func statxTimestampFromDentry(ns int64) linux.StatxTimestamp { } } -// Preconditions: fs.interop != InteropModeShared. +// Preconditions: d.cachedMetadataAuthoritative() == true. func (d *dentry) touchAtime(mnt *vfs.Mount) { if mnt.Flags.NoATime { return @@ -51,8 +51,8 @@ func (d *dentry) touchAtime(mnt *vfs.Mount) { mnt.EndWrite() } -// Preconditions: fs.interop != InteropModeShared. The caller has successfully -// called vfs.Mount.CheckBeginWrite(). +// Preconditions: d.cachedMetadataAuthoritative() == true. The caller has +// successfully called vfs.Mount.CheckBeginWrite(). func (d *dentry) touchCtime() { now := d.fs.clock.Now().Nanoseconds() d.metadataMu.Lock() @@ -60,8 +60,8 @@ func (d *dentry) touchCtime() { d.metadataMu.Unlock() } -// Preconditions: fs.interop != InteropModeShared. The caller has successfully -// called vfs.Mount.CheckBeginWrite(). +// Preconditions: d.cachedMetadataAuthoritative() == true. The caller has +// successfully called vfs.Mount.CheckBeginWrite(). func (d *dentry) touchCMtime() { now := d.fs.clock.Now().Nanoseconds() d.metadataMu.Lock() @@ -70,6 +70,8 @@ func (d *dentry) touchCMtime() { d.metadataMu.Unlock() } +// Preconditions: d.cachedMetadataAuthoritative() == true. The caller has +// locked d.metadataMu. func (d *dentry) touchCMtimeLocked() { now := d.fs.clock.Now().Nanoseconds() atomic.StoreInt64(&d.mtime, now) diff --git a/pkg/sentry/fsimpl/tmpfs/filesystem.go b/pkg/sentry/fsimpl/tmpfs/filesystem.go index ce0dbb154..ed40f6b52 100644 --- a/pkg/sentry/fsimpl/tmpfs/filesystem.go +++ b/pkg/sentry/fsimpl/tmpfs/filesystem.go @@ -410,7 +410,7 @@ func (d *dentry) open(ctx context.Context, rp *vfs.ResolvingPath, opts *vfs.Open if err := fd.vfsfd.Init(&fd, opts.Flags, rp.Mount(), &d.vfsd, &vfs.FileDescriptionOptions{AllowDirectIO: true}); err != nil { return nil, err } - if opts.Flags&linux.O_TRUNC != 0 { + if !afterCreate && opts.Flags&linux.O_TRUNC != 0 { if _, err := impl.truncate(0); err != nil { return nil, err } -- cgit v1.2.3 From 4fa1c304a133297bc6895729d74aa35a015e759e Mon Sep 17 00:00:00 2001 From: Jamie Liu Date: Fri, 7 Aug 2020 00:10:35 -0700 Subject: Try to update atime and mtime on VFS2 gofer files on dentry eviction. PiperOrigin-RevId: 325388385 --- pkg/sentry/fsimpl/gofer/gofer.go | 150 +++++++++++++++++++++++---------------- pkg/sentry/fsimpl/gofer/time.go | 3 + 2 files changed, 91 insertions(+), 62 deletions(-) (limited to 'pkg/sentry/fsimpl/gofer/time.go') diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go index 6ae796c6d..59323086d 100644 --- a/pkg/sentry/fsimpl/gofer/gofer.go +++ b/pkg/sentry/fsimpl/gofer/gofer.go @@ -192,10 +192,14 @@ const ( // // - File timestamps are based on client clocks. This ensures that users of // the client observe timestamps that are coherent with their own clocks - // and consistent with Linux's semantics. However, since it is not always - // possible for clients to set arbitrary atimes and mtimes, and never - // possible for clients to set arbitrary ctimes, file timestamp changes are - // stored in the client only and never sent to the remote filesystem. + // and consistent with Linux's semantics (in particular, it is not always + // possible for clients to set arbitrary atimes and mtimes depending on the + // remote filesystem implementation, and never possible for clients to set + // arbitrary ctimes.) If a dentry containing a client-defined atime or + // mtime is evicted from cache, client timestamps will be sent to the + // remote filesystem on a best-effort basis to attempt to ensure that + // timestamps will be preserved when another dentry representing the same + // file is instantiated. InteropModeExclusive InteropMode = iota // InteropModeWritethrough is appropriate when there are read-only users of @@ -621,6 +625,12 @@ type dentry struct { // File size, protected by both metadataMu and dataMu (i.e. both must be // locked to mutate it; locking either is sufficient to access it). size uint64 + // If this dentry does not represent a synthetic file, deleted is 0, and + // atimeDirty/mtimeDirty are non-zero, atime/mtime may have diverged from the + // remote file's timestamps, which should be updated when this dentry is + // evicted. + atimeDirty uint32 + mtimeDirty uint32 // nlink counts the number of hard links to this dentry. It's updated and // accessed using atomic operations. It's not protected by metadataMu like the @@ -801,10 +811,12 @@ func (d *dentry) updateFromP9AttrsLocked(mask p9.AttrMask, attr *p9.Attr) { if attr.BlockSize != 0 { atomic.StoreUint32(&d.blockSize, uint32(attr.BlockSize)) } - if mask.ATime { + // Don't override newer client-defined timestamps with old server-defined + // ones. + if mask.ATime && atomic.LoadUint32(&d.atimeDirty) == 0 { atomic.StoreInt64(&d.atime, dentryTimestampFromP9(attr.ATimeSeconds, attr.ATimeNanoSeconds)) } - if mask.MTime { + if mask.MTime && atomic.LoadUint32(&d.mtimeDirty) == 0 { atomic.StoreInt64(&d.mtime, dentryTimestampFromP9(attr.MTimeSeconds, attr.MTimeNanoSeconds)) } if mask.CTime { @@ -901,51 +913,44 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs return err } defer mnt.EndWrite() - setLocalAtime := false - setLocalMtime := false + + if stat.Mask&linux.STATX_SIZE != 0 { + // Reject attempts to truncate files other than regular files, since + // filesystem implementations may return the wrong errno. + switch mode.FileType() { + case linux.S_IFREG: + // ok + case linux.S_IFDIR: + return syserror.EISDIR + default: + return syserror.EINVAL + } + } + + var now int64 if d.cachedMetadataAuthoritative() { - // Timestamp updates will be handled locally. - setLocalAtime = stat.Mask&linux.STATX_ATIME != 0 - setLocalMtime = stat.Mask&linux.STATX_MTIME != 0 - stat.Mask &^= linux.STATX_ATIME | linux.STATX_MTIME - - // Prepare for truncate. - if stat.Mask&linux.STATX_SIZE != 0 { - switch mode.FileType() { - case linux.ModeRegular: - if !setLocalMtime { - // Truncate updates mtime. - setLocalMtime = true - stat.Mtime.Nsec = linux.UTIME_NOW - } - case linux.ModeDirectory: - return syserror.EISDIR - default: - return syserror.EINVAL + // Truncate updates mtime. + if stat.Mask&(linux.STATX_SIZE|linux.STATX_MTIME) == linux.STATX_SIZE { + stat.Mask |= linux.STATX_MTIME + stat.Mtime = linux.StatxTimestamp{ + Nsec: linux.UTIME_NOW, } } + + // Use client clocks for timestamps. + now = d.fs.clock.Now().Nanoseconds() + if stat.Mask&linux.STATX_ATIME != 0 && stat.Atime.Nsec == linux.UTIME_NOW { + stat.Atime = statxTimestampFromDentry(now) + } + if stat.Mask&linux.STATX_MTIME != 0 && stat.Mtime.Nsec == linux.UTIME_NOW { + stat.Mtime = statxTimestampFromDentry(now) + } } + d.metadataMu.Lock() defer d.metadataMu.Unlock() - if stat.Mask&linux.STATX_SIZE != 0 { - // The size needs to be changed even when - // !d.cachedMetadataAuthoritative() because d.mappings has to be - // updated. - d.updateFileSizeLocked(stat.Size) - } if !d.isSynthetic() { if stat.Mask != 0 { - if stat.Mask&linux.STATX_SIZE != 0 { - // Check whether to allow a truncate request to be made. - switch d.mode & linux.S_IFMT { - case linux.S_IFREG: - // Allow. - case linux.S_IFDIR: - return syserror.EISDIR - default: - return syserror.EINVAL - } - } if err := d.file.setAttr(ctx, p9.SetAttrMask{ Permissions: stat.Mask&linux.STATX_MODE != 0, UID: stat.Mask&linux.STATX_UID != 0, @@ -967,6 +972,11 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs }); err != nil { return err } + if stat.Mask&linux.STATX_SIZE != 0 { + // Privatized copy-on-write mappings of truncated pages need to + // be invalidated even if InteropModeShared is in effect. + d.updateFileSizeLocked(stat.Size) + } } if d.fs.opts.interop == InteropModeShared { // There's no point to updating d's metadata in this case since @@ -976,7 +986,6 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs return nil } } - now := d.fs.clock.Now().Nanoseconds() if stat.Mask&linux.STATX_MODE != 0 { atomic.StoreUint32(&d.mode, d.fileType()|uint32(stat.Mode)) } @@ -986,23 +995,18 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs if stat.Mask&linux.STATX_GID != 0 { atomic.StoreUint32(&d.gid, stat.GID) } - if setLocalAtime { - if stat.Atime.Nsec == linux.UTIME_NOW { - atomic.StoreInt64(&d.atime, now) - } else { - atomic.StoreInt64(&d.atime, dentryTimestampFromStatx(stat.Atime)) - } - // Restore mask bits that we cleared earlier. - stat.Mask |= linux.STATX_ATIME - } - if setLocalMtime { - if stat.Mtime.Nsec == linux.UTIME_NOW { - atomic.StoreInt64(&d.mtime, now) - } else { - atomic.StoreInt64(&d.mtime, dentryTimestampFromStatx(stat.Mtime)) - } - // Restore mask bits that we cleared earlier. - stat.Mask |= linux.STATX_MTIME + // Note that stat.Atime.Nsec and stat.Mtime.Nsec can't be UTIME_NOW because + // if d.cachedMetadataAuthoritative() then we converted stat.Atime and + // stat.Mtime to client-local timestamps above, and if + // !d.cachedMetadataAuthoritative() then we returned after calling + // d.file.setAttr(). For the same reason, now must have been initialized. + if stat.Mask&linux.STATX_ATIME != 0 { + atomic.StoreInt64(&d.atime, dentryTimestampFromStatx(stat.Atime)) + atomic.StoreUint32(&d.atimeDirty, 0) + } + if stat.Mask&linux.STATX_MTIME != 0 { + atomic.StoreInt64(&d.mtime, dentryTimestampFromStatx(stat.Mtime)) + atomic.StoreUint32(&d.mtimeDirty, 0) } atomic.StoreInt64(&d.ctime, now) return nil @@ -1248,7 +1252,7 @@ func (d *dentry) destroyLocked(ctx context.Context) { // Write dirty pages back to the remote filesystem. if d.handleWritable { if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, mf, d.handle.writeFromBlocksAt); err != nil { - log.Warningf("gofer.dentry.DecRef: failed to write dirty data back: %v", err) + log.Warningf("gofer.dentry.destroyLocked: failed to write dirty data back: %v", err) } } // Discard cached data. @@ -1261,6 +1265,28 @@ func (d *dentry) destroyLocked(ctx context.Context) { d.handleMu.Unlock() if !d.file.isNil() { + if !d.isDeleted() { + // Write dirty timestamps back to the remote filesystem. + atimeDirty := atomic.LoadUint32(&d.atimeDirty) != 0 + mtimeDirty := atomic.LoadUint32(&d.mtimeDirty) != 0 + if atimeDirty || mtimeDirty { + atime := atomic.LoadInt64(&d.atime) + mtime := atomic.LoadInt64(&d.mtime) + if err := d.file.setAttr(ctx, p9.SetAttrMask{ + ATime: atimeDirty, + ATimeNotSystemTime: atimeDirty, + MTime: mtimeDirty, + MTimeNotSystemTime: mtimeDirty, + }, p9.SetAttr{ + ATimeSeconds: uint64(atime / 1e9), + ATimeNanoSeconds: uint64(atime % 1e9), + MTimeSeconds: uint64(mtime / 1e9), + MTimeNanoSeconds: uint64(mtime % 1e9), + }); err != nil { + log.Warningf("gofer.dentry.destroyLocked: failed to write dirty timestamps back: %v", err) + } + } + } d.file.close(ctx) d.file = p9file{} // Remove d from the set of syncable dentries. diff --git a/pkg/sentry/fsimpl/gofer/time.go b/pkg/sentry/fsimpl/gofer/time.go index 0eef4e16e..2cb8191b9 100644 --- a/pkg/sentry/fsimpl/gofer/time.go +++ b/pkg/sentry/fsimpl/gofer/time.go @@ -47,6 +47,7 @@ func (d *dentry) touchAtime(mnt *vfs.Mount) { now := d.fs.clock.Now().Nanoseconds() d.metadataMu.Lock() atomic.StoreInt64(&d.atime, now) + atomic.StoreUint32(&d.atimeDirty, 1) d.metadataMu.Unlock() mnt.EndWrite() } @@ -67,6 +68,7 @@ func (d *dentry) touchCMtime() { d.metadataMu.Lock() atomic.StoreInt64(&d.mtime, now) atomic.StoreInt64(&d.ctime, now) + atomic.StoreUint32(&d.mtimeDirty, 1) d.metadataMu.Unlock() } @@ -76,4 +78,5 @@ func (d *dentry) touchCMtimeLocked() { now := d.fs.clock.Now().Nanoseconds() atomic.StoreInt64(&d.mtime, now) atomic.StoreInt64(&d.ctime, now) + atomic.StoreUint32(&d.mtimeDirty, 1) } -- cgit v1.2.3