diff options
Diffstat (limited to 'pkg/sentry/fsimpl')
31 files changed, 1196 insertions, 522 deletions
diff --git a/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go b/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go index 1a294b64c..6383dd133 100644 --- a/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go +++ b/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go @@ -147,6 +147,8 @@ func (l *lineDiscipline) StateFields() []string { "outQueue", "termios", "column", + "masterWaiter", + "replicaWaiter", } } @@ -154,17 +156,13 @@ func (l *lineDiscipline) beforeSave() {} func (l *lineDiscipline) StateSave(stateSinkObject state.Sink) { l.beforeSave() - if !state.IsZeroValue(&l.masterWaiter) { - state.Failf("masterWaiter is %#v, expected zero", &l.masterWaiter) - } - if !state.IsZeroValue(&l.replicaWaiter) { - state.Failf("replicaWaiter is %#v, expected zero", &l.replicaWaiter) - } stateSinkObject.Save(0, &l.size) stateSinkObject.Save(1, &l.inQueue) stateSinkObject.Save(2, &l.outQueue) stateSinkObject.Save(3, &l.termios) stateSinkObject.Save(4, &l.column) + stateSinkObject.Save(5, &l.masterWaiter) + stateSinkObject.Save(6, &l.replicaWaiter) } func (l *lineDiscipline) afterLoad() {} @@ -175,6 +173,8 @@ func (l *lineDiscipline) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(2, &l.outQueue) stateSourceObject.Load(3, &l.termios) stateSourceObject.Load(4, &l.column) + stateSourceObject.Load(5, &l.masterWaiter) + stateSourceObject.Load(6, &l.replicaWaiter) } func (o *outputQueueTransformer) StateTypeName() string { diff --git a/pkg/sentry/fsimpl/devpts/line_discipline.go b/pkg/sentry/fsimpl/devpts/line_discipline.go index e6b0e81cf..ae95fdd08 100644 --- a/pkg/sentry/fsimpl/devpts/line_discipline.go +++ b/pkg/sentry/fsimpl/devpts/line_discipline.go @@ -100,10 +100,10 @@ type lineDiscipline struct { column int // masterWaiter is used to wait on the master end of the TTY. - masterWaiter waiter.Queue `state:"zerovalue"` + masterWaiter waiter.Queue // replicaWaiter is used to wait on the replica end of the TTY. - replicaWaiter waiter.Queue `state:"zerovalue"` + replicaWaiter waiter.Queue } func newLineDiscipline(termios linux.KernelTermios) *lineDiscipline { diff --git a/pkg/sentry/fsimpl/devtmpfs/devtmpfs_state_autogen.go b/pkg/sentry/fsimpl/devtmpfs/devtmpfs_state_autogen.go index 86033019a..87f77d00c 100644 --- a/pkg/sentry/fsimpl/devtmpfs/devtmpfs_state_autogen.go +++ b/pkg/sentry/fsimpl/devtmpfs/devtmpfs_state_autogen.go @@ -27,12 +27,11 @@ func (fst *FilesystemType) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(2, &fst.root) } -func (fst *FilesystemType) afterLoad() {} - func (fst *FilesystemType) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &fst.initErr) stateSourceObject.Load(1, &fst.fs) stateSourceObject.Load(2, &fst.root) + stateSourceObject.AfterLoad(fst.afterLoad) } func init() { diff --git a/pkg/sentry/fsimpl/devtmpfs/save_restore.go b/pkg/sentry/fsimpl/devtmpfs/save_restore.go new file mode 100644 index 000000000..28832d850 --- /dev/null +++ b/pkg/sentry/fsimpl/devtmpfs/save_restore.go @@ -0,0 +1,23 @@ +// 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 devtmpfs + +// afterLoad is invoked by stateify. +func (fst *FilesystemType) afterLoad() { + if fst.fs != nil { + // Ensure that we don't create another filesystem. + fst.initOnce.Do(func() {}) + } +} diff --git a/pkg/sentry/fsimpl/eventfd/eventfd.go b/pkg/sentry/fsimpl/eventfd/eventfd.go index 1c27ad700..5b29f2358 100644 --- a/pkg/sentry/fsimpl/eventfd/eventfd.go +++ b/pkg/sentry/fsimpl/eventfd/eventfd.go @@ -43,7 +43,7 @@ type EventFileDescription struct { // queue is used to notify interested parties when the event object // becomes readable or writable. - queue waiter.Queue `state:"zerovalue"` + queue waiter.Queue // mu protects the fields below. mu sync.Mutex `state:"nosave"` diff --git a/pkg/sentry/fsimpl/eventfd/eventfd_state_autogen.go b/pkg/sentry/fsimpl/eventfd/eventfd_state_autogen.go index 1c19f900a..3af030742 100644 --- a/pkg/sentry/fsimpl/eventfd/eventfd_state_autogen.go +++ b/pkg/sentry/fsimpl/eventfd/eventfd_state_autogen.go @@ -16,6 +16,7 @@ func (efd *EventFileDescription) StateFields() []string { "FileDescriptionDefaultImpl", "DentryMetadataFileDescriptionImpl", "NoLockFD", + "queue", "val", "semMode", "hostfd", @@ -26,16 +27,14 @@ func (efd *EventFileDescription) beforeSave() {} func (efd *EventFileDescription) StateSave(stateSinkObject state.Sink) { efd.beforeSave() - if !state.IsZeroValue(&efd.queue) { - state.Failf("queue is %#v, expected zero", &efd.queue) - } stateSinkObject.Save(0, &efd.vfsfd) stateSinkObject.Save(1, &efd.FileDescriptionDefaultImpl) stateSinkObject.Save(2, &efd.DentryMetadataFileDescriptionImpl) stateSinkObject.Save(3, &efd.NoLockFD) - stateSinkObject.Save(4, &efd.val) - stateSinkObject.Save(5, &efd.semMode) - stateSinkObject.Save(6, &efd.hostfd) + stateSinkObject.Save(4, &efd.queue) + stateSinkObject.Save(5, &efd.val) + stateSinkObject.Save(6, &efd.semMode) + stateSinkObject.Save(7, &efd.hostfd) } func (efd *EventFileDescription) afterLoad() {} @@ -45,9 +44,10 @@ func (efd *EventFileDescription) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(1, &efd.FileDescriptionDefaultImpl) stateSourceObject.Load(2, &efd.DentryMetadataFileDescriptionImpl) stateSourceObject.Load(3, &efd.NoLockFD) - stateSourceObject.Load(4, &efd.val) - stateSourceObject.Load(5, &efd.semMode) - stateSourceObject.Load(6, &efd.hostfd) + stateSourceObject.Load(4, &efd.queue) + stateSourceObject.Load(5, &efd.val) + stateSourceObject.Load(6, &efd.semMode) + stateSourceObject.Load(7, &efd.hostfd) } func init() { diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go index 18c884b59..c3af30f49 100644 --- a/pkg/sentry/fsimpl/gofer/directory.go +++ b/pkg/sentry/fsimpl/gofer/directory.go @@ -16,7 +16,6 @@ package gofer import ( "fmt" - "sync" "sync/atomic" "gvisor.dev/gvisor/pkg/abi/linux" @@ -26,6 +25,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/kernel/pipe" "gvisor.dev/gvisor/pkg/sentry/socket/unix/transport" "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/usermem" ) @@ -92,7 +92,7 @@ func (d *dentry) createSyntheticChildLocked(opts *createSyntheticOpts) { child := &dentry{ refs: 1, // held by d fs: d.fs, - ino: d.fs.nextSyntheticIno(), + ino: d.fs.nextIno(), mode: uint32(opts.mode), uid: uint32(opts.kuid), gid: uint32(opts.kgid), @@ -235,7 +235,7 @@ func (d *dentry) getDirents(ctx context.Context) ([]vfs.Dirent, error) { } dirent := vfs.Dirent{ Name: p9d.Name, - Ino: uint64(inoFromPath(p9d.QID.Path)), + Ino: d.fs.inoFromQIDPath(p9d.QID.Path), NextOff: int64(len(dirents) + 1), } // p9 does not expose 9P2000.U's DMDEVICE, DMNAMEDPIPE, or diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go index 94d96261b..baecb88c4 100644 --- a/pkg/sentry/fsimpl/gofer/filesystem.go +++ b/pkg/sentry/fsimpl/gofer/filesystem.go @@ -35,7 +35,7 @@ import ( // Sync implements vfs.FilesystemImpl.Sync. func (fs *filesystem) Sync(ctx context.Context) error { - // Snapshot current syncable dentries and special files. + // Snapshot current syncable dentries and special file FDs. fs.syncMu.Lock() ds := make([]*dentry, 0, len(fs.syncableDentries)) for d := range fs.syncableDentries { @@ -53,22 +53,28 @@ func (fs *filesystem) Sync(ctx context.Context) error { // regardless. var retErr error - // Sync regular files. + // Sync syncable dentries. for _, d := range ds { - err := d.syncCachedFile(ctx) + err := d.syncCachedFile(ctx, true /* forFilesystemSync */) d.DecRef(ctx) - if err != nil && retErr == nil { - retErr = err + if err != nil { + ctx.Infof("gofer.filesystem.Sync: dentry.syncCachedFile failed: %v", err) + if retErr == nil { + retErr = err + } } } // Sync special files, which may be writable but do not use dentry shared // handles (so they won't be synced by the above). for _, sffd := range sffds { - err := sffd.Sync(ctx) + err := sffd.sync(ctx, true /* forFilesystemSync */) sffd.vfsfd.DecRef(ctx) - if err != nil && retErr == nil { - retErr = err + if err != nil { + ctx.Infof("gofer.filesystem.Sync: specialFileFD.sync failed: %v", err) + if retErr == nil { + retErr = err + } } } @@ -229,7 +235,7 @@ func (fs *filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.Vir return nil, err } if child != nil { - if !file.isNil() && inoFromPath(qid.Path) == child.ino { + 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) @@ -1512,7 +1518,6 @@ func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath d.IncRef() return &endpoint{ dentry: d, - file: d.file.file, path: opts.Addr, }, nil } @@ -1591,7 +1596,3 @@ func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDe defer fs.renameMu.RUnlock() return genericPrependPath(vfsroot, vd.Mount(), vd.Dentry().Impl().(*dentry), b) } - -func (fs *filesystem) nextSyntheticIno() inodeNumber { - return inodeNumber(atomic.AddUint64(&fs.syntheticSeq, 1) | syntheticInoMask) -} diff --git a/pkg/sentry/fsimpl/gofer/fstree.go b/pkg/sentry/fsimpl/gofer/fstree.go index 507dc82e6..6e43d4a4b 100644 --- a/pkg/sentry/fsimpl/gofer/fstree.go +++ b/pkg/sentry/fsimpl/gofer/fstree.go @@ -34,7 +34,7 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *dentry, b if mnt == vfsroot.Mount() && &d.vfsd == vfsroot.Dentry() { return vfs.PrependPathAtVFSRootError{} } - if &d.vfsd == mnt.Root() { + if mnt != nil && &d.vfsd == mnt.Root() { return nil } if d.parent == nil { @@ -44,3 +44,12 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *dentry, b d = d.parent } } + +// DebugPathname returns a pathname to d relative to its filesystem root. +// DebugPathname does not correspond to any Linux function; it's used to +// generate dentry pathnames for debugging. +func genericDebugPathname(d *dentry) string { + var b fspath.Builder + _ = genericPrependPath(vfs.VirtualDentry{}, nil, d, &b) + return b.String() +} diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go index 8e179539c..f7c94cce1 100644 --- a/pkg/sentry/fsimpl/gofer/gofer.go +++ b/pkg/sentry/fsimpl/gofer/gofer.go @@ -26,6 +26,9 @@ // *** "memmap.Mappable locks taken by Translate" below this point // dentry.handleMu // dentry.dataMu +// filesystem.inoMu +// specialFileFD.mu +// specialFileFD.bufMu // // Locking dentry.dirMu in multiple dentries requires that either ancestor // dentries are locked before descendant dentries, or that filesystem.renameMu @@ -36,7 +39,6 @@ import ( "fmt" "strconv" "strings" - "sync" "sync/atomic" "syscall" @@ -53,6 +55,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/pgalloc" "gvisor.dev/gvisor/pkg/sentry/socket/unix/transport" "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/unet" "gvisor.dev/gvisor/pkg/usermem" @@ -81,7 +84,7 @@ type filesystem struct { iopts InternalFilesystemOptions // client is the client used by this filesystem. client is immutable. - client *p9.Client `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. + client *p9.Client `state:"nosave"` // clock is a realtime clock used to set timestamps in file operations. clock ktime.Clock @@ -89,6 +92,9 @@ type filesystem struct { // devMinor is the filesystem's minor device number. devMinor is immutable. devMinor uint32 + // root is the root dentry. root is immutable. + root *dentry + // renameMu serves two purposes: // // - It synchronizes path resolution with renaming initiated by this @@ -108,34 +114,26 @@ type filesystem struct { cachedDentries dentryList cachedDentriesLen uint64 - // syncableDentries contains all dentries in this filesystem for which - // !dentry.file.isNil(). specialFileFDs contains all open specialFileFDs. - // These fields are protected by syncMu. + // syncableDentries contains all non-synthetic dentries. specialFileFDs + // contains all open specialFileFDs. These fields are protected by syncMu. syncMu sync.Mutex `state:"nosave"` syncableDentries map[*dentry]struct{} specialFileFDs map[*specialFileFD]struct{} - // syntheticSeq stores a counter to used to generate unique inodeNumber for - // synthetic dentries. - syntheticSeq uint64 -} - -// inodeNumber represents inode number reported in Dirent.Ino. For regular -// dentries, it comes from QID.Path from the 9P server. Synthetic dentries -// have have their inodeNumber generated sequentially, with the MSB reserved to -// prevent conflicts with regular dentries. -// -// +stateify savable -type inodeNumber uint64 + // inoByQIDPath maps previously-observed QID.Paths to inode numbers + // assigned to those paths. inoByQIDPath is not preserved across + // checkpoint/restore because QIDs may be reused between different gofer + // processes, so QIDs may be repeated for different files across + // checkpoint/restore. inoByQIDPath is protected by inoMu. + inoMu sync.Mutex `state:"nosave"` + inoByQIDPath map[uint64]uint64 `state:"nosave"` -// Reserve MSB for synthetic mounts. -const syntheticInoMask = uint64(1) << 63 + // lastIno is the last inode number assigned to a file. lastIno is accessed + // using atomic memory operations. + lastIno uint64 -func inoFromPath(path uint64) inodeNumber { - if path&syntheticInoMask != 0 { - log.Warningf("Dropping MSB from ino, collision is possible. Original: %d, new: %d", path, path&^syntheticInoMask) - } - return inodeNumber(path &^ syntheticInoMask) + // savedDentryRW records open read/write handles during save/restore. + savedDentryRW map[*dentry]savedDentryRW } // +stateify savable @@ -247,6 +245,10 @@ const ( // // +stateify savable type InternalFilesystemOptions struct { + // If UniqueID is non-empty, it is an opaque string used to reassociate the + // filesystem with a new server FD during restoration from checkpoint. + UniqueID string + // If LeakConnection is true, do not close the connection to the server // when the Filesystem is released. This is necessary for deployments in // which servers can handle only a single client and report failure if that @@ -286,46 +288,11 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt mopts := vfs.GenericParseMountOptions(opts.Data) var fsopts filesystemOptions - // Check that the transport is "fd". - trans, ok := mopts["trans"] - if !ok { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: transport must be specified as 'trans=fd'") - return nil, nil, syserror.EINVAL - } - delete(mopts, "trans") - if trans != "fd" { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: unsupported transport: trans=%s", trans) - return nil, nil, syserror.EINVAL - } - - // Check that read and write FDs are provided and identical. - rfdstr, ok := mopts["rfdno"] - if !ok { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: read FD must be specified as 'rfdno=<file descriptor>") - return nil, nil, syserror.EINVAL - } - delete(mopts, "rfdno") - rfd, err := strconv.Atoi(rfdstr) - if err != nil { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid read FD: rfdno=%s", rfdstr) - return nil, nil, syserror.EINVAL - } - wfdstr, ok := mopts["wfdno"] - if !ok { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: write FD must be specified as 'wfdno=<file descriptor>") - return nil, nil, syserror.EINVAL - } - delete(mopts, "wfdno") - wfd, err := strconv.Atoi(wfdstr) + fd, err := getFDFromMountOptionsMap(ctx, mopts) if err != nil { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid write FD: wfdno=%s", wfdstr) - return nil, nil, syserror.EINVAL - } - if rfd != wfd { - ctx.Warningf("gofer.FilesystemType.GetFilesystem: read FD (%d) and write FD (%d) must be equal", rfd, wfd) - return nil, nil, syserror.EINVAL + return nil, nil, err } - fsopts.fd = rfd + fsopts.fd = fd // Get the attach name. fsopts.aname = "/" @@ -441,57 +408,44 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt } // If !ok, iopts being the zero value is correct. - // Establish a connection with the server. - conn, err := unet.NewSocket(fsopts.fd) + // Construct the filesystem object. + devMinor, err := vfsObj.GetAnonBlockDevMinor() if err != nil { return nil, nil, err } + fs := &filesystem{ + mfp: mfp, + opts: fsopts, + iopts: iopts, + clock: ktime.RealtimeClockFromContext(ctx), + devMinor: devMinor, + syncableDentries: make(map[*dentry]struct{}), + specialFileFDs: make(map[*specialFileFD]struct{}), + inoByQIDPath: make(map[uint64]uint64), + } + fs.vfsfs.Init(vfsObj, &fstype, fs) - // Perform version negotiation with the server. - ctx.UninterruptibleSleepStart(false) - client, err := p9.NewClient(conn, fsopts.msize, fsopts.version) - ctx.UninterruptibleSleepFinish(false) - if err != nil { - conn.Close() + // Connect to the server. + if err := fs.dial(ctx); err != nil { return nil, nil, err } - // Ownership of conn has been transferred to client. // Perform attach to obtain the filesystem root. ctx.UninterruptibleSleepStart(false) - attached, err := client.Attach(fsopts.aname) + attached, err := fs.client.Attach(fsopts.aname) ctx.UninterruptibleSleepFinish(false) if err != nil { - client.Close() + fs.vfsfs.DecRef(ctx) return nil, nil, err } attachFile := p9file{attached} qid, attrMask, attr, err := attachFile.getAttr(ctx, dentryAttrMask()) if err != nil { attachFile.close(ctx) - client.Close() + fs.vfsfs.DecRef(ctx) return nil, nil, err } - // Construct the filesystem object. - devMinor, err := vfsObj.GetAnonBlockDevMinor() - if err != nil { - attachFile.close(ctx) - client.Close() - return nil, nil, err - } - fs := &filesystem{ - mfp: mfp, - opts: fsopts, - iopts: iopts, - client: client, - clock: ktime.RealtimeClockFromContext(ctx), - devMinor: devMinor, - syncableDentries: make(map[*dentry]struct{}), - specialFileFDs: make(map[*specialFileFD]struct{}), - } - fs.vfsfs.Init(vfsObj, &fstype, fs) - // Construct the root dentry. root, err := fs.newDentry(ctx, attachFile, qid, attrMask, &attr) if err != nil { @@ -504,10 +458,72 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt // being "cached" and subsequently evicted. Its resources will still be // cleaned up by fs.Release(). root.refs = 2 + fs.root = root return &fs.vfsfs, &root.vfsd, nil } +func getFDFromMountOptionsMap(ctx context.Context, mopts map[string]string) (int, error) { + // Check that the transport is "fd". + trans, ok := mopts["trans"] + if !ok || trans != "fd" { + ctx.Warningf("gofer.getFDFromMountOptionsMap: transport must be specified as 'trans=fd'") + return -1, syserror.EINVAL + } + delete(mopts, "trans") + + // Check that read and write FDs are provided and identical. + rfdstr, ok := mopts["rfdno"] + if !ok { + ctx.Warningf("gofer.getFDFromMountOptionsMap: read FD must be specified as 'rfdno=<file descriptor>'") + return -1, syserror.EINVAL + } + delete(mopts, "rfdno") + rfd, err := strconv.Atoi(rfdstr) + if err != nil { + ctx.Warningf("gofer.getFDFromMountOptionsMap: invalid read FD: rfdno=%s", rfdstr) + return -1, syserror.EINVAL + } + wfdstr, ok := mopts["wfdno"] + if !ok { + ctx.Warningf("gofer.getFDFromMountOptionsMap: write FD must be specified as 'wfdno=<file descriptor>'") + return -1, syserror.EINVAL + } + delete(mopts, "wfdno") + wfd, err := strconv.Atoi(wfdstr) + if err != nil { + ctx.Warningf("gofer.getFDFromMountOptionsMap: invalid write FD: wfdno=%s", wfdstr) + return -1, syserror.EINVAL + } + if rfd != wfd { + ctx.Warningf("gofer.getFDFromMountOptionsMap: read FD (%d) and write FD (%d) must be equal", rfd, wfd) + return -1, syserror.EINVAL + } + return rfd, nil +} + +// Preconditions: fs.client == nil. +func (fs *filesystem) dial(ctx context.Context) error { + // Establish a connection with the server. + conn, err := unet.NewSocket(fs.opts.fd) + if err != nil { + return err + } + + // Perform version negotiation with the server. + ctx.UninterruptibleSleepStart(false) + client, err := p9.NewClient(conn, fs.opts.msize, fs.opts.version) + ctx.UninterruptibleSleepFinish(false) + if err != nil { + conn.Close() + return err + } + // Ownership of conn has been transferred to client. + + fs.client = client + return nil +} + // Release implements vfs.FilesystemImpl.Release. func (fs *filesystem) Release(ctx context.Context) { mf := fs.mfp.MemoryFile() @@ -574,12 +590,15 @@ type dentry struct { // filesystem.renameMu. name string + // qidPath is the p9.QID.Path for this file. qidPath is immutable. + qidPath uint64 + // file is the unopened p9.File that backs this dentry. file is immutable. // // If file.isNil(), this dentry represents a synthetic file, i.e. a file // that does not exist on the remote filesystem. As of this writing, the // only files that can be synthetic are sockets, pipes, and directories. - file p9file `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. + file p9file `state:"nosave"` // If deleted is non-zero, the file represented by this dentry has been // deleted. deleted is accessed using atomic memory operations. @@ -623,12 +642,12 @@ type dentry struct { // To mutate: // - Lock metadataMu and use atomic operations to update because we might // have atomic readers that don't hold the lock. - metadataMu sync.Mutex `state:"nosave"` - ino inodeNumber // immutable - mode uint32 // type is immutable, perms are mutable - uid uint32 // auth.KUID, but stored as raw uint32 for sync/atomic - gid uint32 // auth.KGID, but ... - blockSize uint32 // 0 if unknown + metadataMu sync.Mutex `state:"nosave"` + ino uint64 // immutable + mode uint32 // type is immutable, perms are mutable + uid uint32 // auth.KUID, but stored as raw uint32 for sync/atomic + gid uint32 // auth.KGID, but ... + blockSize uint32 // 0 if unknown // Timestamps, all nsecs from the Unix epoch. atime int64 mtime int64 @@ -679,9 +698,9 @@ type dentry struct { // (isNil() == false), it may be mutated with handleMu locked, but cannot // be closed until the dentry is destroyed. handleMu sync.RWMutex `state:"nosave"` - readFile p9file `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. - writeFile p9file `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. - hostFD int32 + readFile p9file `state:"nosave"` + writeFile p9file `state:"nosave"` + hostFD int32 `state:"nosave"` dataMu sync.RWMutex `state:"nosave"` @@ -758,8 +777,9 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma d := &dentry{ fs: fs, + qidPath: qid.Path, file: file, - ino: inoFromPath(qid.Path), + ino: fs.inoFromQIDPath(qid.Path), mode: uint32(attr.Mode), uid: uint32(fs.opts.dfltuid), gid: uint32(fs.opts.dfltgid), @@ -802,6 +822,21 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma return d, nil } +func (fs *filesystem) inoFromQIDPath(qidPath uint64) uint64 { + fs.inoMu.Lock() + defer fs.inoMu.Unlock() + if ino, ok := fs.inoByQIDPath[qidPath]; ok { + return ino + } + ino := fs.nextIno() + fs.inoByQIDPath[qidPath] = ino + return ino +} + +func (fs *filesystem) nextIno() uint64 { + return atomic.AddUint64(&fs.lastIno, 1) +} + func (d *dentry) isSynthetic() bool { return d.file.isNil() } @@ -853,7 +888,7 @@ func (d *dentry) updateFromP9AttrsLocked(mask p9.AttrMask, attr *p9.Attr) { } } -// Preconditions: !d.isSynthetic() +// Preconditions: !d.isSynthetic(). func (d *dentry) updateFromGetattr(ctx context.Context) error { // Use d.readFile or d.writeFile, which represent 9P fids that have been // opened, in preference to d.file, which represents a 9P fid that has not. @@ -1269,33 +1304,40 @@ func (d *dentry) checkCachingLocked(ctx context.Context) { d.fs.cachedDentriesLen++ d.cached = true if d.fs.cachedDentriesLen > d.fs.opts.maxCachedDentries { - victim := d.fs.cachedDentries.Back() - d.fs.cachedDentries.Remove(victim) - d.fs.cachedDentriesLen-- - victim.cached = false - // victim.refs may have become non-zero from an earlier path resolution - // since it was inserted into fs.cachedDentries. - if atomic.LoadInt64(&victim.refs) == 0 { - if victim.parent != nil { - victim.parent.dirMu.Lock() - if !victim.vfsd.IsDead() { - // Note that victim can't be a mount point (in any mount - // namespace), since VFS holds references on mount points. - d.fs.vfsfs.VirtualFilesystem().InvalidateDentry(ctx, &victim.vfsd) - delete(victim.parent.children, victim.name) - // We're only deleting the dentry, not the file it - // represents, so we don't need to update - // victimParent.dirents etc. - } - victim.parent.dirMu.Unlock() - } - victim.destroyLocked(ctx) - } + d.fs.evictCachedDentryLocked(ctx) // Whether or not victim was destroyed, we brought fs.cachedDentriesLen // back down to fs.opts.maxCachedDentries, so we don't loop. } } +// Preconditions: +// * fs.renameMu must be locked for writing; it may be temporarily unlocked. +// * fs.cachedDentriesLen != 0. +func (fs *filesystem) evictCachedDentryLocked(ctx context.Context) { + victim := fs.cachedDentries.Back() + fs.cachedDentries.Remove(victim) + fs.cachedDentriesLen-- + victim.cached = false + // victim.refs may have become non-zero from an earlier path resolution + // since it was inserted into fs.cachedDentries. + if atomic.LoadInt64(&victim.refs) == 0 { + if victim.parent != nil { + victim.parent.dirMu.Lock() + if !victim.vfsd.IsDead() { + // Note that victim can't be a mount point (in any mount + // namespace), since VFS holds references on mount points. + fs.vfsfs.VirtualFilesystem().InvalidateDentry(ctx, &victim.vfsd) + delete(victim.parent.children, victim.name) + // We're only deleting the dentry, not the file it + // represents, so we don't need to update + // victimParent.dirents etc. + } + victim.parent.dirMu.Unlock() + } + victim.destroyLocked(ctx) + } +} + // destroyLocked destroys the dentry. // // Preconditions: @@ -1623,6 +1665,33 @@ func (d *dentry) syncRemoteFileLocked(ctx context.Context) error { return nil } +func (d *dentry) syncCachedFile(ctx context.Context, forFilesystemSync bool) error { + d.handleMu.RLock() + defer d.handleMu.RUnlock() + h := d.writeHandleLocked() + if h.isOpen() { + // Write back dirty pages to the remote file. + d.dataMu.Lock() + err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, d.fs.mfp.MemoryFile(), h.writeFromBlocksAt) + d.dataMu.Unlock() + if err != nil { + return err + } + } + if err := d.syncRemoteFileLocked(ctx); err != nil { + if !forFilesystemSync { + return err + } + // Only return err if we can reasonably have expected sync to succeed + // (d is a regular file and was opened for writing). + if d.isRegularFile() && h.isOpen() { + return err + } + ctx.Debugf("gofer.dentry.syncCachedFile: syncing non-writable or non-regular-file dentry failed: %v", err) + } + return nil +} + // incLinks increments link count. func (d *dentry) incLinks() { if atomic.LoadUint32(&d.nlink) == 0 { @@ -1650,7 +1719,7 @@ type fileDescription struct { vfs.FileDescriptionDefaultImpl vfs.LockFD - lockLogging sync.Once `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. + lockLogging sync.Once `state:"nosave"` } func (fd *fileDescription) filesystem() *filesystem { diff --git a/pkg/sentry/fsimpl/gofer/gofer_state_autogen.go b/pkg/sentry/fsimpl/gofer/gofer_state_autogen.go index 4d8560b53..289c3d6d4 100644 --- a/pkg/sentry/fsimpl/gofer/gofer_state_autogen.go +++ b/pkg/sentry/fsimpl/gofer/gofer_state_autogen.go @@ -121,11 +121,13 @@ func (fs *filesystem) StateFields() []string { "iopts", "clock", "devMinor", + "root", "cachedDentries", "cachedDentriesLen", "syncableDentries", "specialFileFDs", - "syntheticSeq", + "lastIno", + "savedDentryRW", } } @@ -139,11 +141,13 @@ func (fs *filesystem) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(3, &fs.iopts) stateSinkObject.Save(4, &fs.clock) stateSinkObject.Save(5, &fs.devMinor) - stateSinkObject.Save(6, &fs.cachedDentries) - stateSinkObject.Save(7, &fs.cachedDentriesLen) - stateSinkObject.Save(8, &fs.syncableDentries) - stateSinkObject.Save(9, &fs.specialFileFDs) - stateSinkObject.Save(10, &fs.syntheticSeq) + stateSinkObject.Save(6, &fs.root) + stateSinkObject.Save(7, &fs.cachedDentries) + stateSinkObject.Save(8, &fs.cachedDentriesLen) + stateSinkObject.Save(9, &fs.syncableDentries) + stateSinkObject.Save(10, &fs.specialFileFDs) + stateSinkObject.Save(11, &fs.lastIno) + stateSinkObject.Save(12, &fs.savedDentryRW) } func (fs *filesystem) afterLoad() {} @@ -155,19 +159,13 @@ func (fs *filesystem) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(3, &fs.iopts) stateSourceObject.Load(4, &fs.clock) stateSourceObject.Load(5, &fs.devMinor) - stateSourceObject.Load(6, &fs.cachedDentries) - stateSourceObject.Load(7, &fs.cachedDentriesLen) - stateSourceObject.Load(8, &fs.syncableDentries) - stateSourceObject.Load(9, &fs.specialFileFDs) - stateSourceObject.Load(10, &fs.syntheticSeq) -} - -func (i *inodeNumber) StateTypeName() string { - return "pkg/sentry/fsimpl/gofer.inodeNumber" -} - -func (i *inodeNumber) StateFields() []string { - return nil + stateSourceObject.Load(6, &fs.root) + stateSourceObject.Load(7, &fs.cachedDentries) + stateSourceObject.Load(8, &fs.cachedDentriesLen) + stateSourceObject.Load(9, &fs.syncableDentries) + stateSourceObject.Load(10, &fs.specialFileFDs) + stateSourceObject.Load(11, &fs.lastIno) + stateSourceObject.Load(12, &fs.savedDentryRW) } func (f *filesystemOptions) StateTypeName() string { @@ -240,6 +238,7 @@ func (i *InternalFilesystemOptions) StateTypeName() string { func (i *InternalFilesystemOptions) StateFields() []string { return []string{ + "UniqueID", "LeakConnection", "OpenSocketsByConnecting", } @@ -249,15 +248,17 @@ func (i *InternalFilesystemOptions) beforeSave() {} func (i *InternalFilesystemOptions) StateSave(stateSinkObject state.Sink) { i.beforeSave() - stateSinkObject.Save(0, &i.LeakConnection) - stateSinkObject.Save(1, &i.OpenSocketsByConnecting) + stateSinkObject.Save(0, &i.UniqueID) + stateSinkObject.Save(1, &i.LeakConnection) + stateSinkObject.Save(2, &i.OpenSocketsByConnecting) } func (i *InternalFilesystemOptions) afterLoad() {} func (i *InternalFilesystemOptions) StateLoad(stateSourceObject state.Source) { - stateSourceObject.Load(0, &i.LeakConnection) - stateSourceObject.Load(1, &i.OpenSocketsByConnecting) + stateSourceObject.Load(0, &i.UniqueID) + stateSourceObject.Load(1, &i.LeakConnection) + stateSourceObject.Load(2, &i.OpenSocketsByConnecting) } func (d *dentry) StateTypeName() string { @@ -271,6 +272,7 @@ func (d *dentry) StateFields() []string { "fs", "parent", "name", + "qidPath", "deleted", "cached", "dentryEntry", @@ -291,7 +293,6 @@ func (d *dentry) StateFields() []string { "mtimeDirty", "nlink", "mappings", - "hostFD", "cache", "dirty", "pf", @@ -304,8 +305,6 @@ func (d *dentry) StateFields() []string { } } -func (d *dentry) beforeSave() {} - func (d *dentry) StateSave(stateSinkObject state.Sink) { d.beforeSave() stateSinkObject.Save(0, &d.vfsd) @@ -313,27 +312,27 @@ func (d *dentry) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(2, &d.fs) stateSinkObject.Save(3, &d.parent) stateSinkObject.Save(4, &d.name) - stateSinkObject.Save(5, &d.deleted) - stateSinkObject.Save(6, &d.cached) - stateSinkObject.Save(7, &d.dentryEntry) - stateSinkObject.Save(8, &d.children) - stateSinkObject.Save(9, &d.syntheticChildren) - stateSinkObject.Save(10, &d.dirents) - stateSinkObject.Save(11, &d.ino) - stateSinkObject.Save(12, &d.mode) - stateSinkObject.Save(13, &d.uid) - stateSinkObject.Save(14, &d.gid) - stateSinkObject.Save(15, &d.blockSize) - stateSinkObject.Save(16, &d.atime) - stateSinkObject.Save(17, &d.mtime) - stateSinkObject.Save(18, &d.ctime) - stateSinkObject.Save(19, &d.btime) - stateSinkObject.Save(20, &d.size) - stateSinkObject.Save(21, &d.atimeDirty) - stateSinkObject.Save(22, &d.mtimeDirty) - stateSinkObject.Save(23, &d.nlink) - stateSinkObject.Save(24, &d.mappings) - stateSinkObject.Save(25, &d.hostFD) + stateSinkObject.Save(5, &d.qidPath) + stateSinkObject.Save(6, &d.deleted) + stateSinkObject.Save(7, &d.cached) + stateSinkObject.Save(8, &d.dentryEntry) + stateSinkObject.Save(9, &d.children) + stateSinkObject.Save(10, &d.syntheticChildren) + stateSinkObject.Save(11, &d.dirents) + stateSinkObject.Save(12, &d.ino) + stateSinkObject.Save(13, &d.mode) + stateSinkObject.Save(14, &d.uid) + stateSinkObject.Save(15, &d.gid) + stateSinkObject.Save(16, &d.blockSize) + stateSinkObject.Save(17, &d.atime) + stateSinkObject.Save(18, &d.mtime) + stateSinkObject.Save(19, &d.ctime) + stateSinkObject.Save(20, &d.btime) + stateSinkObject.Save(21, &d.size) + stateSinkObject.Save(22, &d.atimeDirty) + stateSinkObject.Save(23, &d.mtimeDirty) + stateSinkObject.Save(24, &d.nlink) + stateSinkObject.Save(25, &d.mappings) stateSinkObject.Save(26, &d.cache) stateSinkObject.Save(27, &d.dirty) stateSinkObject.Save(28, &d.pf) @@ -345,35 +344,33 @@ func (d *dentry) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(34, &d.watches) } -func (d *dentry) afterLoad() {} - func (d *dentry) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &d.vfsd) stateSourceObject.Load(1, &d.refs) stateSourceObject.Load(2, &d.fs) stateSourceObject.Load(3, &d.parent) stateSourceObject.Load(4, &d.name) - stateSourceObject.Load(5, &d.deleted) - stateSourceObject.Load(6, &d.cached) - stateSourceObject.Load(7, &d.dentryEntry) - stateSourceObject.Load(8, &d.children) - stateSourceObject.Load(9, &d.syntheticChildren) - stateSourceObject.Load(10, &d.dirents) - stateSourceObject.Load(11, &d.ino) - stateSourceObject.Load(12, &d.mode) - stateSourceObject.Load(13, &d.uid) - stateSourceObject.Load(14, &d.gid) - stateSourceObject.Load(15, &d.blockSize) - stateSourceObject.Load(16, &d.atime) - stateSourceObject.Load(17, &d.mtime) - stateSourceObject.Load(18, &d.ctime) - stateSourceObject.Load(19, &d.btime) - stateSourceObject.Load(20, &d.size) - stateSourceObject.Load(21, &d.atimeDirty) - stateSourceObject.Load(22, &d.mtimeDirty) - stateSourceObject.Load(23, &d.nlink) - stateSourceObject.Load(24, &d.mappings) - stateSourceObject.Load(25, &d.hostFD) + stateSourceObject.Load(5, &d.qidPath) + stateSourceObject.Load(6, &d.deleted) + stateSourceObject.Load(7, &d.cached) + stateSourceObject.Load(8, &d.dentryEntry) + stateSourceObject.Load(9, &d.children) + stateSourceObject.Load(10, &d.syntheticChildren) + stateSourceObject.Load(11, &d.dirents) + stateSourceObject.Load(12, &d.ino) + stateSourceObject.Load(13, &d.mode) + stateSourceObject.Load(14, &d.uid) + stateSourceObject.Load(15, &d.gid) + stateSourceObject.Load(16, &d.blockSize) + stateSourceObject.Load(17, &d.atime) + stateSourceObject.Load(18, &d.mtime) + stateSourceObject.Load(19, &d.ctime) + stateSourceObject.Load(20, &d.btime) + stateSourceObject.Load(21, &d.size) + stateSourceObject.Load(22, &d.atimeDirty) + stateSourceObject.Load(23, &d.mtimeDirty) + stateSourceObject.Load(24, &d.nlink) + stateSourceObject.Load(25, &d.mappings) stateSourceObject.Load(26, &d.cache) stateSourceObject.Load(27, &d.dirty) stateSourceObject.Load(28, &d.pf) @@ -383,6 +380,7 @@ func (d *dentry) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(32, &d.pipe) stateSourceObject.Load(33, &d.locks) stateSourceObject.Load(34, &d.watches) + stateSourceObject.AfterLoad(d.afterLoad) } func (fd *fileDescription) StateTypeName() string { @@ -461,12 +459,37 @@ func (d *dentryPlatformFile) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(2, &d.hostFileMapper) } -func (d *dentryPlatformFile) afterLoad() {} - func (d *dentryPlatformFile) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &d.dentry) stateSourceObject.Load(1, &d.fdRefs) stateSourceObject.Load(2, &d.hostFileMapper) + stateSourceObject.AfterLoad(d.afterLoad) +} + +func (s *savedDentryRW) StateTypeName() string { + return "pkg/sentry/fsimpl/gofer.savedDentryRW" +} + +func (s *savedDentryRW) StateFields() []string { + return []string{ + "read", + "write", + } +} + +func (s *savedDentryRW) beforeSave() {} + +func (s *savedDentryRW) StateSave(stateSinkObject state.Sink) { + s.beforeSave() + stateSinkObject.Save(0, &s.read) + stateSinkObject.Save(1, &s.write) +} + +func (s *savedDentryRW) afterLoad() {} + +func (s *savedDentryRW) StateLoad(stateSourceObject state.Source) { + stateSourceObject.Load(0, &s.read) + stateSourceObject.Load(1, &s.write) } func (e *endpoint) StateTypeName() string { @@ -504,9 +527,10 @@ func (fd *specialFileFD) StateFields() []string { "fileDescription", "isRegularFile", "seekable", - "haveQueue", "queue", "off", + "haveBuf", + "buf", } } @@ -517,20 +541,21 @@ func (fd *specialFileFD) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(0, &fd.fileDescription) stateSinkObject.Save(1, &fd.isRegularFile) stateSinkObject.Save(2, &fd.seekable) - stateSinkObject.Save(3, &fd.haveQueue) - stateSinkObject.Save(4, &fd.queue) - stateSinkObject.Save(5, &fd.off) + stateSinkObject.Save(3, &fd.queue) + stateSinkObject.Save(4, &fd.off) + stateSinkObject.Save(5, &fd.haveBuf) + stateSinkObject.Save(6, &fd.buf) } -func (fd *specialFileFD) afterLoad() {} - func (fd *specialFileFD) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &fd.fileDescription) stateSourceObject.Load(1, &fd.isRegularFile) stateSourceObject.Load(2, &fd.seekable) - stateSourceObject.Load(3, &fd.haveQueue) - stateSourceObject.Load(4, &fd.queue) - stateSourceObject.Load(5, &fd.off) + stateSourceObject.Load(3, &fd.queue) + stateSourceObject.Load(4, &fd.off) + stateSourceObject.Load(5, &fd.haveBuf) + stateSourceObject.Load(6, &fd.buf) + stateSourceObject.AfterLoad(fd.afterLoad) } func init() { @@ -539,7 +564,6 @@ func init() { state.Register((*directoryFD)(nil)) state.Register((*FilesystemType)(nil)) state.Register((*filesystem)(nil)) - state.Register((*inodeNumber)(nil)) state.Register((*filesystemOptions)(nil)) state.Register((*InteropMode)(nil)) state.Register((*InternalFilesystemOptions)(nil)) @@ -547,6 +571,7 @@ func init() { state.Register((*fileDescription)(nil)) state.Register((*regularFileFD)(nil)) state.Register((*dentryPlatformFile)(nil)) + state.Register((*savedDentryRW)(nil)) state.Register((*endpoint)(nil)) state.Register((*specialFileFD)(nil)) } diff --git a/pkg/sentry/fsimpl/gofer/host_named_pipe.go b/pkg/sentry/fsimpl/gofer/host_named_pipe.go index 7294de7d6..c7bf10007 100644 --- a/pkg/sentry/fsimpl/gofer/host_named_pipe.go +++ b/pkg/sentry/fsimpl/gofer/host_named_pipe.go @@ -51,8 +51,24 @@ func blockUntilNonblockingPipeHasWriter(ctx context.Context, fd int32) error { if ok { return nil } - if err := sleepBetweenNamedPipeOpenChecks(ctx); err != nil { - return err + if sleepErr := sleepBetweenNamedPipeOpenChecks(ctx); sleepErr != nil { + // Another application thread may have opened this pipe for + // writing, succeeded because we previously opened the pipe for + // reading, and subsequently interrupted us for checkpointing (e.g. + // this occurs in mknod tests under cooperative save/restore). In + // this case, our open has to succeed for the checkpoint to include + // a readable FD for the pipe, which is in turn necessary to + // restore the other thread's writable FD for the same pipe + // (otherwise it will get ENXIO). So we have to check + // nonblockingPipeHasWriter() once last time. + ok, err := nonblockingPipeHasWriter(fd) + if err != nil { + return err + } + if ok { + return nil + } + return sleepErr } } } diff --git a/pkg/sentry/fsimpl/gofer/regular_file.go b/pkg/sentry/fsimpl/gofer/regular_file.go index f8b19bae7..dc8a890cb 100644 --- a/pkg/sentry/fsimpl/gofer/regular_file.go +++ b/pkg/sentry/fsimpl/gofer/regular_file.go @@ -18,7 +18,6 @@ import ( "fmt" "io" "math" - "sync" "sync/atomic" "gvisor.dev/gvisor/pkg/abi/linux" @@ -31,6 +30,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/pgalloc" "gvisor.dev/gvisor/pkg/sentry/usage" "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/usermem" ) @@ -624,23 +624,7 @@ func regularFileSeekLocked(ctx context.Context, d *dentry, fdOffset, offset int6 // Sync implements vfs.FileDescriptionImpl.Sync. func (fd *regularFileFD) Sync(ctx context.Context) error { - return fd.dentry().syncCachedFile(ctx) -} - -func (d *dentry) syncCachedFile(ctx context.Context) error { - d.handleMu.RLock() - defer d.handleMu.RUnlock() - - if h := d.writeHandleLocked(); h.isOpen() { - d.dataMu.Lock() - // Write dirty cached data to the remote file. - err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, d.fs.mfp.MemoryFile(), h.writeFromBlocksAt) - d.dataMu.Unlock() - if err != nil { - return err - } - } - return d.syncRemoteFileLocked(ctx) + return fd.dentry().syncCachedFile(ctx, false /* lowSyncExpectations */) } // ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap. @@ -913,7 +897,7 @@ type dentryPlatformFile struct { hostFileMapper fsutil.HostFileMapper // hostFileMapperInitOnce is used to lazily initialize hostFileMapper. - hostFileMapperInitOnce sync.Once `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. + hostFileMapperInitOnce sync.Once `state:"nosave"` } // IncRef implements memmap.File.IncRef. diff --git a/pkg/sentry/fsimpl/gofer/save_restore.go b/pkg/sentry/fsimpl/gofer/save_restore.go new file mode 100644 index 000000000..e995619a6 --- /dev/null +++ b/pkg/sentry/fsimpl/gofer/save_restore.go @@ -0,0 +1,327 @@ +// 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 gofer + +import ( + "fmt" + "io" + "sync/atomic" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/fdnotifier" + "gvisor.dev/gvisor/pkg/p9" + "gvisor.dev/gvisor/pkg/safemem" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +type saveRestoreContextID int + +const ( + // CtxRestoreServerFDMap is a Context.Value key for a map[string]int + // mapping filesystem unique IDs (cf. InternalFilesystemOptions.UniqueID) + // to host FDs. + CtxRestoreServerFDMap saveRestoreContextID = iota +) + +// +stateify savable +type savedDentryRW struct { + read bool + write bool +} + +// PreprareSave implements vfs.FilesystemImplSaveRestoreExtension.PrepareSave. +func (fs *filesystem) PrepareSave(ctx context.Context) error { + if len(fs.iopts.UniqueID) == 0 { + return fmt.Errorf("gofer.filesystem with no UniqueID cannot be saved") + } + + // Purge cached dentries, which may not be reopenable after restore due to + // permission changes. + fs.renameMu.Lock() + for fs.cachedDentriesLen != 0 { + fs.evictCachedDentryLocked(ctx) + } + fs.renameMu.Unlock() + + // Buffer pipe data so that it's available for reading after restore. (This + // is a legacy VFS1 feature.) + fs.syncMu.Lock() + for sffd := range fs.specialFileFDs { + if sffd.dentry().fileType() == linux.S_IFIFO && sffd.vfsfd.IsReadable() { + if err := sffd.savePipeData(ctx); err != nil { + fs.syncMu.Unlock() + return err + } + } + } + fs.syncMu.Unlock() + + // Flush local state to the remote filesystem. + if err := fs.Sync(ctx); err != nil { + return err + } + + fs.savedDentryRW = make(map[*dentry]savedDentryRW) + return fs.root.prepareSaveRecursive(ctx) +} + +// Preconditions: +// * fd represents a pipe. +// * fd is readable. +func (fd *specialFileFD) savePipeData(ctx context.Context) error { + fd.bufMu.Lock() + defer fd.bufMu.Unlock() + var buf [usermem.PageSize]byte + for { + n, err := fd.handle.readToBlocksAt(ctx, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf[:])), ^uint64(0)) + if n != 0 { + fd.buf = append(fd.buf, buf[:n]...) + } + if err != nil { + if err == io.EOF || err == syserror.EAGAIN { + break + } + return err + } + } + if len(fd.buf) != 0 { + atomic.StoreUint32(&fd.haveBuf, 1) + } + return nil +} + +func (d *dentry) prepareSaveRecursive(ctx context.Context) error { + if d.isRegularFile() && !d.cachedMetadataAuthoritative() { + // Get updated metadata for d in case we need to perform metadata + // validation during restore. + if err := d.updateFromGetattr(ctx); err != nil { + return err + } + } + if !d.readFile.isNil() || !d.writeFile.isNil() { + d.fs.savedDentryRW[d] = savedDentryRW{ + read: !d.readFile.isNil(), + write: !d.writeFile.isNil(), + } + } + d.dirMu.Lock() + defer d.dirMu.Unlock() + for _, child := range d.children { + if child != nil { + if err := child.prepareSaveRecursive(ctx); err != nil { + return err + } + } + } + return nil +} + +// beforeSave is invoked by stateify. +func (d *dentry) beforeSave() { + if d.vfsd.IsDead() { + panic(fmt.Sprintf("gofer.dentry(%q).beforeSave: deleted and invalidated dentries can't be restored", genericDebugPathname(d))) + } +} + +// afterLoad is invoked by stateify. +func (d *dentry) afterLoad() { + d.hostFD = -1 +} + +// afterLoad is invoked by stateify. +func (d *dentryPlatformFile) afterLoad() { + if d.hostFileMapper.IsInited() { + // Ensure that we don't call d.hostFileMapper.Init() again. + d.hostFileMapperInitOnce.Do(func() {}) + } +} + +// afterLoad is invoked by stateify. +func (fd *specialFileFD) afterLoad() { + fd.handle.fd = -1 +} + +// CompleteRestore implements +// vfs.FilesystemImplSaveRestoreExtension.CompleteRestore. +func (fs *filesystem) CompleteRestore(ctx context.Context, opts vfs.CompleteRestoreOptions) error { + fdmapv := ctx.Value(CtxRestoreServerFDMap) + if fdmapv == nil { + return fmt.Errorf("no server FD map available") + } + fdmap := fdmapv.(map[string]int) + fd, ok := fdmap[fs.iopts.UniqueID] + if !ok { + return fmt.Errorf("no server FD available for filesystem with unique ID %q", fs.iopts.UniqueID) + } + fs.opts.fd = fd + if err := fs.dial(ctx); err != nil { + return err + } + fs.inoByQIDPath = make(map[uint64]uint64) + + // Restore the filesystem root. + ctx.UninterruptibleSleepStart(false) + attached, err := fs.client.Attach(fs.opts.aname) + ctx.UninterruptibleSleepFinish(false) + if err != nil { + return err + } + attachFile := p9file{attached} + qid, attrMask, attr, err := attachFile.getAttr(ctx, dentryAttrMask()) + if err != nil { + return err + } + if err := fs.root.restoreFile(ctx, attachFile, qid, attrMask, &attr, &opts); err != nil { + return err + } + + // Restore remaining dentries. + if err := fs.root.restoreDescendantsRecursive(ctx, &opts); err != nil { + return err + } + + // Re-open handles for specialFileFDs. Unlike the initial open + // (dentry.openSpecialFile()), pipes are always opened without blocking; + // non-readable pipe FDs are opened last to ensure that they don't get + // ENXIO if another specialFileFD represents the read end of the same pipe. + // This is consistent with VFS1. + haveWriteOnlyPipes := false + for fd := range fs.specialFileFDs { + if fd.dentry().fileType() == linux.S_IFIFO && !fd.vfsfd.IsReadable() { + haveWriteOnlyPipes = true + continue + } + if err := fd.completeRestore(ctx); err != nil { + return err + } + } + if haveWriteOnlyPipes { + for fd := range fs.specialFileFDs { + if fd.dentry().fileType() == linux.S_IFIFO && !fd.vfsfd.IsReadable() { + if err := fd.completeRestore(ctx); err != nil { + return err + } + } + } + } + + // Discard state only required during restore. + fs.savedDentryRW = nil + + return nil +} + +func (d *dentry) restoreFile(ctx context.Context, file p9file, qid p9.QID, attrMask p9.AttrMask, attr *p9.Attr, opts *vfs.CompleteRestoreOptions) error { + d.file = file + + // Gofers do not preserve QID across checkpoint/restore, so: + // + // - We must assume that the remote filesystem did not change in a way that + // would invalidate dentries, since we can't revalidate dentries by + // checking QIDs. + // + // - We need to associate the new QID.Path with the existing d.ino. + d.qidPath = qid.Path + d.fs.inoMu.Lock() + d.fs.inoByQIDPath[qid.Path] = d.ino + d.fs.inoMu.Unlock() + + // Check metadata stability before updating metadata. + d.metadataMu.Lock() + defer d.metadataMu.Unlock() + if d.isRegularFile() { + if opts.ValidateFileSizes { + if !attrMask.Size { + return fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: file size not available", genericDebugPathname(d)) + } + if d.size != attr.Size { + return fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: size changed from %d to %d", genericDebugPathname(d), d.size, attr.Size) + } + } + if opts.ValidateFileModificationTimestamps { + if !attrMask.MTime { + return fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime not available", genericDebugPathname(d)) + } + if want := dentryTimestampFromP9(attr.MTimeSeconds, attr.MTimeNanoSeconds); d.mtime != want { + return fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime changed from %+v to %+v", genericDebugPathname(d), linux.NsecToStatxTimestamp(d.mtime), linux.NsecToStatxTimestamp(want)) + } + } + } + if !d.cachedMetadataAuthoritative() { + d.updateFromP9AttrsLocked(attrMask, attr) + } + + if rw, ok := d.fs.savedDentryRW[d]; ok { + if err := d.ensureSharedHandle(ctx, rw.read, rw.write, false /* trunc */); err != nil { + return err + } + } + + return nil +} + +// Preconditions: d is not synthetic. +func (d *dentry) restoreDescendantsRecursive(ctx context.Context, opts *vfs.CompleteRestoreOptions) error { + for _, child := range d.children { + if child == nil { + continue + } + if _, ok := d.fs.syncableDentries[child]; !ok { + // child is synthetic. + continue + } + if err := child.restoreRecursive(ctx, opts); err != nil { + return err + } + } + return nil +} + +// Preconditions: d is not synthetic (but note that since this function +// restores d.file, d.file.isNil() is always true at this point, so this can +// only be detected by checking filesystem.syncableDentries). d.parent has been +// restored. +func (d *dentry) restoreRecursive(ctx context.Context, opts *vfs.CompleteRestoreOptions) error { + qid, file, attrMask, attr, err := d.parent.file.walkGetAttrOne(ctx, d.name) + if err != nil { + return err + } + if err := d.restoreFile(ctx, file, qid, attrMask, &attr, opts); err != nil { + return err + } + return d.restoreDescendantsRecursive(ctx, opts) +} + +func (fd *specialFileFD) completeRestore(ctx context.Context) error { + d := fd.dentry() + h, err := openHandle(ctx, d.file, fd.vfsfd.IsReadable(), fd.vfsfd.IsWritable(), false /* trunc */) + if err != nil { + return err + } + fd.handle = h + + ftype := d.fileType() + fd.haveQueue = (ftype == linux.S_IFIFO || ftype == linux.S_IFSOCK) && fd.handle.fd >= 0 + if fd.haveQueue { + if err := fdnotifier.AddFD(fd.handle.fd, &fd.queue); err != nil { + return err + } + } + + return nil +} diff --git a/pkg/sentry/fsimpl/gofer/socket.go b/pkg/sentry/fsimpl/gofer/socket.go index 326b940a7..a21199eac 100644 --- a/pkg/sentry/fsimpl/gofer/socket.go +++ b/pkg/sentry/fsimpl/gofer/socket.go @@ -42,9 +42,6 @@ type endpoint struct { // dentry is the filesystem dentry which produced this endpoint. dentry *dentry - // file is the p9 file that contains a single unopened fid. - file p9.File `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. - // path is the sentry path where this endpoint is bound. path string } @@ -116,7 +113,7 @@ func (e *endpoint) UnidirectionalConnect(ctx context.Context) (transport.Connect } func (e *endpoint) newConnectedEndpoint(ctx context.Context, flags p9.ConnectFlags, queue *waiter.Queue) (*host.SCMConnectedEndpoint, *syserr.Error) { - hostFile, err := e.file.Connect(flags) + hostFile, err := e.dentry.file.connect(ctx, flags) if err != nil { return nil, syserr.ErrConnectionRefused } @@ -131,7 +128,7 @@ func (e *endpoint) newConnectedEndpoint(ctx context.Context, flags p9.ConnectFla c, serr := host.NewSCMEndpoint(ctx, hostFD, queue, e.path) if serr != nil { - log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v flags %+v: %v", e.file, flags, serr) + log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v flags %+v: %v", e.dentry.file, flags, serr) return nil, serr } return c, nil diff --git a/pkg/sentry/fsimpl/gofer/special_file.go b/pkg/sentry/fsimpl/gofer/special_file.go index 71581736c..625400c0b 100644 --- a/pkg/sentry/fsimpl/gofer/special_file.go +++ b/pkg/sentry/fsimpl/gofer/special_file.go @@ -15,7 +15,6 @@ package gofer import ( - "sync" "sync/atomic" "syscall" @@ -25,6 +24,7 @@ import ( "gvisor.dev/gvisor/pkg/p9" "gvisor.dev/gvisor/pkg/safemem" "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/usermem" "gvisor.dev/gvisor/pkg/waiter" @@ -40,7 +40,7 @@ type specialFileFD struct { fileDescription // handle is used for file I/O. handle is immutable. - handle handle `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. + handle handle `state:"nosave"` // isRegularFile is true if this FD represents a regular file which is only // possible when filesystemOptions.regularFilesUseSpecialFileFD is in @@ -54,12 +54,20 @@ type specialFileFD struct { // haveQueue is true if this file description represents a file for which // queue may send I/O readiness events. haveQueue is immutable. - haveQueue bool + haveQueue bool `state:"nosave"` queue waiter.Queue // If seekable is true, off is the file offset. off is protected by mu. mu sync.Mutex `state:"nosave"` off int64 + + // If haveBuf is non-zero, this FD represents a pipe, and buf contains data + // read from the pipe from previous calls to specialFileFD.savePipeData(). + // haveBuf and buf are protected by bufMu. haveBuf is accessed using atomic + // memory operations. + bufMu sync.Mutex `state:"nosave"` + haveBuf uint32 + buf []byte } func newSpecialFileFD(h handle, mnt *vfs.Mount, d *dentry, locks *vfs.FileLocks, flags uint32) (*specialFileFD, error) { @@ -87,6 +95,9 @@ func newSpecialFileFD(h handle, mnt *vfs.Mount, d *dentry, locks *vfs.FileLocks, } return nil, err } + d.fs.syncMu.Lock() + d.fs.specialFileFDs[fd] = struct{}{} + d.fs.syncMu.Unlock() return fd, nil } @@ -161,26 +172,51 @@ func (fd *specialFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs return 0, syserror.EOPNOTSUPP } - // Going through dst.CopyOutFrom() holds MM locks around file operations of - // unknown duration. For regularFileFD, doing so is necessary to support - // mmap due to lock ordering; MM locks precede dentry.dataMu. That doesn't - // hold here since specialFileFD doesn't client-cache data. Just buffer the - // read instead. if d := fd.dentry(); d.cachedMetadataAuthoritative() { d.touchAtime(fd.vfsfd.Mount()) } + + bufN := int64(0) + if atomic.LoadUint32(&fd.haveBuf) != 0 { + var err error + fd.bufMu.Lock() + if len(fd.buf) != 0 { + var n int + n, err = dst.CopyOut(ctx, fd.buf) + dst = dst.DropFirst(n) + fd.buf = fd.buf[n:] + if len(fd.buf) == 0 { + atomic.StoreUint32(&fd.haveBuf, 0) + fd.buf = nil + } + bufN = int64(n) + if offset >= 0 { + offset += bufN + } + } + fd.bufMu.Unlock() + if err != nil { + return bufN, err + } + } + + // Going through dst.CopyOutFrom() would hold MM locks around file + // operations of unknown duration. For regularFileFD, doing so is necessary + // to support mmap due to lock ordering; MM locks precede dentry.dataMu. + // That doesn't hold here since specialFileFD doesn't client-cache data. + // Just buffer the read instead. buf := make([]byte, dst.NumBytes()) n, err := fd.handle.readToBlocksAt(ctx, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf)), uint64(offset)) if err == syserror.EAGAIN { err = syserror.ErrWouldBlock } if n == 0 { - return 0, err + return bufN, err } if cp, cperr := dst.CopyOut(ctx, buf[:n]); cperr != nil { - return int64(cp), cperr + return bufN + int64(cp), cperr } - return int64(n), err + return bufN + int64(n), err } // Read implements vfs.FileDescriptionImpl.Read. @@ -217,16 +253,16 @@ func (fd *specialFileFD) pwrite(ctx context.Context, src usermem.IOSequence, off } d := fd.dentry() - // If the regular file fd was opened with O_APPEND, make sure the file size - // is updated. There is a possible race here if size is modified externally - // after metadata cache is updated. - if fd.isRegularFile && fd.vfsfd.StatusFlags()&linux.O_APPEND != 0 && !d.cachedMetadataAuthoritative() { - if err := d.updateFromGetattr(ctx); err != nil { - return 0, offset, err + if fd.isRegularFile { + // If the regular file fd was opened with O_APPEND, make sure the file + // size is updated. There is a possible race here if size is modified + // externally after metadata cache is updated. + if fd.vfsfd.StatusFlags()&linux.O_APPEND != 0 && !d.cachedMetadataAuthoritative() { + if err := d.updateFromGetattr(ctx); err != nil { + return 0, offset, err + } } - } - if fd.isRegularFile { // We need to hold the metadataMu *while* writing to a regular file. d.metadataMu.Lock() defer d.metadataMu.Unlock() @@ -306,13 +342,31 @@ func (fd *specialFileFD) Seek(ctx context.Context, offset int64, whence int32) ( // Sync implements vfs.FileDescriptionImpl.Sync. func (fd *specialFileFD) Sync(ctx context.Context) error { - // If we have a host FD, fsyncing it is likely to be faster than an fsync - // RPC. - if fd.handle.fd >= 0 { - ctx.UninterruptibleSleepStart(false) - err := syscall.Fsync(int(fd.handle.fd)) - ctx.UninterruptibleSleepFinish(false) - return err + return fd.sync(ctx, false /* forFilesystemSync */) +} + +func (fd *specialFileFD) sync(ctx context.Context, forFilesystemSync bool) error { + err := func() error { + // If we have a host FD, fsyncing it is likely to be faster than an fsync + // RPC. + if fd.handle.fd >= 0 { + ctx.UninterruptibleSleepStart(false) + err := syscall.Fsync(int(fd.handle.fd)) + ctx.UninterruptibleSleepFinish(false) + return err + } + return fd.handle.file.fsync(ctx) + }() + if err != nil { + if !forFilesystemSync { + return err + } + // Only return err if we can reasonably have expected sync to succeed + // (fd represents a regular file that was opened for writing). + if fd.isRegularFile && fd.vfsfd.IsWritable() { + return err + } + ctx.Debugf("gofer.specialFileFD.sync: syncing non-writable or non-regular-file FD failed: %v", err) } - return fd.handle.file.fsync(ctx) + return nil } diff --git a/pkg/sentry/fsimpl/host/control.go b/pkg/sentry/fsimpl/host/control.go index 0135e4428..13ef48cb5 100644 --- a/pkg/sentry/fsimpl/host/control.go +++ b/pkg/sentry/fsimpl/host/control.go @@ -79,7 +79,7 @@ func fdsToFiles(ctx context.Context, fds []int) []*vfs.FileDescription { } // Create the file backed by hostFD. - file, err := ImportFD(ctx, kernel.KernelFromContext(ctx).HostMount(), fd, false /* isTTY */) + file, err := NewFD(ctx, kernel.KernelFromContext(ctx).HostMount(), fd, &NewFDOptions{}) if err != nil { ctx.Warningf("Error creating file from host FD: %v", err) break diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go index 698e913fe..eeed0f97d 100644 --- a/pkg/sentry/fsimpl/host/host.go +++ b/pkg/sentry/fsimpl/host/host.go @@ -19,6 +19,7 @@ package host import ( "fmt" "math" + "sync/atomic" "syscall" "golang.org/x/sys/unix" @@ -40,34 +41,106 @@ import ( "gvisor.dev/gvisor/pkg/waiter" ) -func newInode(fs *filesystem, hostFD int, fileType linux.FileMode, isTTY bool) (*inode, error) { - // Determine if hostFD is seekable. If not, this syscall will return ESPIPE - // (see fs/read_write.c:llseek), e.g. for pipes, sockets, and some character - // devices. +// inode implements kernfs.Inode. +// +// +stateify savable +type inode struct { + kernfs.InodeNoStatFS + kernfs.InodeNotDirectory + kernfs.InodeNotSymlink + kernfs.InodeTemporary // This holds no meaning as this inode can't be Looked up and is always valid. + + locks vfs.FileLocks + + // When the reference count reaches zero, the host fd is closed. + inodeRefs + + // hostFD contains the host fd that this file was originally created from, + // which must be available at time of restore. + // + // This field is initialized at creation time and is immutable. + hostFD int + + // ino is an inode number unique within this filesystem. + // + // This field is initialized at creation time and is immutable. + ino uint64 + + // ftype is the file's type (a linux.S_IFMT mask). + // + // This field is initialized at creation time and is immutable. + ftype uint16 + + // mayBlock is true if hostFD is non-blocking, and operations on it may + // return EAGAIN or EWOULDBLOCK instead of blocking. + // + // This field is initialized at creation time and is immutable. + mayBlock bool + + // seekable is false if lseek(hostFD) returns ESPIPE. We assume that file + // offsets are meaningful iff seekable is true. + // + // This field is initialized at creation time and is immutable. + seekable bool + + // isTTY is true if this file represents a TTY. + // + // This field is initialized at creation time and is immutable. + isTTY bool + + // savable is true if hostFD may be saved/restored by its numeric value. + // + // This field is initialized at creation time and is immutable. + savable bool + + // Event queue for blocking operations. + queue waiter.Queue + + // mapsMu protects mappings. + mapsMu sync.Mutex `state:"nosave"` + + // If this file is mmappable, mappings tracks mappings of hostFD into + // memmap.MappingSpaces. + mappings memmap.MappingSet + + // pf implements platform.File for mappings of hostFD. + pf inodePlatformFile + + // If haveBuf is non-zero, hostFD represents a pipe, and buf contains data + // read from the pipe from previous calls to inode.beforeSave(). haveBuf + // and buf are protected by bufMu. haveBuf is accessed using atomic memory + // operations. + bufMu sync.Mutex `state:"nosave"` + haveBuf uint32 + buf []byte +} + +func newInode(ctx context.Context, fs *filesystem, hostFD int, savable bool, fileType linux.FileMode, isTTY bool) (*inode, error) { + // Determine if hostFD is seekable. _, err := unix.Seek(hostFD, 0, linux.SEEK_CUR) seekable := err != syserror.ESPIPE + // We expect regular files to be seekable, as this is required for them to + // be memory-mappable. + if !seekable && fileType == syscall.S_IFREG { + ctx.Infof("host.newInode: host FD %d is a non-seekable regular file", hostFD) + return nil, syserror.ESPIPE + } i := &inode{ - hostFD: hostFD, - ino: fs.NextIno(), - isTTY: isTTY, - wouldBlock: wouldBlock(uint32(fileType)), - seekable: seekable, - // NOTE(b/38213152): Technically, some obscure char devices can be memory - // mapped, but we only allow regular files. - canMap: fileType == linux.S_IFREG, + hostFD: hostFD, + ino: fs.NextIno(), + ftype: uint16(fileType), + mayBlock: fileType != syscall.S_IFREG && fileType != syscall.S_IFDIR, + seekable: seekable, + isTTY: isTTY, + savable: savable, } i.pf.inode = i i.EnableLeakCheck() - // Non-seekable files can't be memory mapped, assert this. - if !i.seekable && i.canMap { - panic("files that can return EWOULDBLOCK (sockets, pipes, etc.) cannot be memory mapped") - } - - // If the hostFD would block, we must set it to non-blocking and handle - // blocking behavior in the sentry. - if i.wouldBlock { + // If the hostFD can return EWOULDBLOCK when set to non-blocking, do so and + // handle blocking behavior in the sentry. + if i.mayBlock { if err := syscall.SetNonblock(i.hostFD, true); err != nil { return nil, err } @@ -80,6 +153,11 @@ func newInode(fs *filesystem, hostFD int, fileType linux.FileMode, isTTY bool) ( // NewFDOptions contains options to NewFD. type NewFDOptions struct { + // If Savable is true, the host file descriptor may be saved/restored by + // numeric value; the sandbox API requires a corresponding host FD with the + // same numeric value to be provieded at time of restore. + Savable bool + // If IsTTY is true, the file descriptor is a TTY. IsTTY bool @@ -114,7 +192,7 @@ func NewFD(ctx context.Context, mnt *vfs.Mount, hostFD int, opts *NewFDOptions) } d := &kernfs.Dentry{} - i, err := newInode(fs, hostFD, linux.FileMode(s.Mode).FileType(), opts.IsTTY) + i, err := newInode(ctx, fs, hostFD, opts.Savable, linux.FileMode(s.Mode).FileType(), opts.IsTTY) if err != nil { return nil, err } @@ -132,7 +210,8 @@ func NewFD(ctx context.Context, mnt *vfs.Mount, hostFD int, opts *NewFDOptions) // ImportFD sets up and returns a vfs.FileDescription from a donated fd. func ImportFD(ctx context.Context, mnt *vfs.Mount, hostFD int, isTTY bool) (*vfs.FileDescription, error) { return NewFD(ctx, mnt, hostFD, &NewFDOptions{ - IsTTY: isTTY, + Savable: true, + IsTTY: isTTY, }) } @@ -191,68 +270,6 @@ func (fs *filesystem) PrependPath(ctx context.Context, vfsroot, vd vfs.VirtualDe return vfs.PrependPathSyntheticError{} } -// inode implements kernfs.Inode. -// -// +stateify savable -type inode struct { - kernfs.InodeNoStatFS - kernfs.InodeNotDirectory - kernfs.InodeNotSymlink - kernfs.InodeTemporary // This holds no meaning as this inode can't be Looked up and is always valid. - - locks vfs.FileLocks - - // When the reference count reaches zero, the host fd is closed. - inodeRefs - - // hostFD contains the host fd that this file was originally created from, - // which must be available at time of restore. - // - // This field is initialized at creation time and is immutable. - hostFD int - - // ino is an inode number unique within this filesystem. - // - // This field is initialized at creation time and is immutable. - ino uint64 - - // isTTY is true if this file represents a TTY. - // - // This field is initialized at creation time and is immutable. - isTTY bool - - // seekable is false if the host fd points to a file representing a stream, - // e.g. a socket or a pipe. Such files are not seekable and can return - // EWOULDBLOCK for I/O operations. - // - // This field is initialized at creation time and is immutable. - seekable bool - - // wouldBlock is true if the host FD would return EWOULDBLOCK for - // operations that would block. - // - // This field is initialized at creation time and is immutable. - wouldBlock bool - - // Event queue for blocking operations. - queue waiter.Queue - - // canMap specifies whether we allow the file to be memory mapped. - // - // This field is initialized at creation time and is immutable. - canMap bool - - // mapsMu protects mappings. - mapsMu sync.Mutex `state:"nosave"` - - // If canMap is true, mappings tracks mappings of hostFD into - // memmap.MappingSpaces. - mappings memmap.MappingSet - - // pf implements platform.File for mappings of hostFD. - pf inodePlatformFile -} - // CheckPermissions implements kernfs.Inode.CheckPermissions. func (i *inode) CheckPermissions(ctx context.Context, creds *auth.Credentials, ats vfs.AccessTypes) error { var s syscall.Stat_t @@ -448,7 +465,7 @@ func (i *inode) SetStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Cre // DecRef implements kernfs.Inode.DecRef. func (i *inode) DecRef(ctx context.Context) { i.inodeRefs.DecRef(func() { - if i.wouldBlock { + if i.mayBlock { fdnotifier.RemoveFD(int32(i.hostFD)) } if err := unix.Close(i.hostFD); err != nil { @@ -567,6 +584,13 @@ func (f *fileDescription) Allocate(ctx context.Context, mode, offset, length uin // PRead implements vfs.FileDescriptionImpl.PRead. func (f *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { + // Check that flags are supported. + // + // TODO(gvisor.dev/issue/2601): Support select preadv2 flags. + if opts.Flags&^linux.RWF_HIPRI != 0 { + return 0, syserror.EOPNOTSUPP + } + i := f.inode if !i.seekable { return 0, syserror.ESPIPE @@ -577,19 +601,31 @@ func (f *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, off // Read implements vfs.FileDescriptionImpl.Read. func (f *fileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { + // Check that flags are supported. + // + // TODO(gvisor.dev/issue/2601): Support select preadv2 flags. + if opts.Flags&^linux.RWF_HIPRI != 0 { + return 0, syserror.EOPNOTSUPP + } + i := f.inode if !i.seekable { + bufN, err := i.readFromBuf(ctx, &dst) + if err != nil { + return bufN, err + } n, err := readFromHostFD(ctx, i.hostFD, dst, -1, opts.Flags) + total := bufN + n if isBlockError(err) { // If we got any data at all, return it as a "completed" partial read // rather than retrying until complete. - if n != 0 { + if total != 0 { err = nil } else { err = syserror.ErrWouldBlock } } - return n, err + return total, err } f.offsetMu.Lock() @@ -599,13 +635,26 @@ func (f *fileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts return n, err } -func readFromHostFD(ctx context.Context, hostFD int, dst usermem.IOSequence, offset int64, flags uint32) (int64, error) { - // Check that flags are supported. - // - // TODO(gvisor.dev/issue/2601): Support select preadv2 flags. - if flags&^linux.RWF_HIPRI != 0 { - return 0, syserror.EOPNOTSUPP +func (i *inode) readFromBuf(ctx context.Context, dst *usermem.IOSequence) (int64, error) { + if atomic.LoadUint32(&i.haveBuf) == 0 { + return 0, nil } + i.bufMu.Lock() + defer i.bufMu.Unlock() + if len(i.buf) == 0 { + return 0, nil + } + n, err := dst.CopyOut(ctx, i.buf) + *dst = dst.DropFirst(n) + i.buf = i.buf[n:] + if len(i.buf) == 0 { + atomic.StoreUint32(&i.haveBuf, 0) + i.buf = nil + } + return int64(n), err +} + +func readFromHostFD(ctx context.Context, hostFD int, dst usermem.IOSequence, offset int64, flags uint32) (int64, error) { reader := hostfd.GetReadWriterAt(int32(hostFD), offset, flags) n, err := dst.CopyOutFrom(ctx, reader) hostfd.PutReadWriterAt(reader) @@ -735,14 +784,16 @@ func (f *fileDescription) Seek(_ context.Context, offset int64, whence int32) (i } // Sync implements vfs.FileDescriptionImpl.Sync. -func (f *fileDescription) Sync(context.Context) error { +func (f *fileDescription) Sync(ctx context.Context) error { // TODO(gvisor.dev/issue/1897): Currently, we always sync everything. return unix.Fsync(f.inode.hostFD) } // ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap. func (f *fileDescription) ConfigureMMap(_ context.Context, opts *memmap.MMapOpts) error { - if !f.inode.canMap { + // NOTE(b/38213152): Technically, some obscure char devices can be memory + // mapped, but we only allow regular files. + if f.inode.ftype != syscall.S_IFREG { return syserror.ENODEV } i := f.inode @@ -753,13 +804,17 @@ func (f *fileDescription) ConfigureMMap(_ context.Context, opts *memmap.MMapOpts // EventRegister implements waiter.Waitable.EventRegister. func (f *fileDescription) EventRegister(e *waiter.Entry, mask waiter.EventMask) { f.inode.queue.EventRegister(e, mask) - fdnotifier.UpdateFD(int32(f.inode.hostFD)) + if f.inode.mayBlock { + fdnotifier.UpdateFD(int32(f.inode.hostFD)) + } } // EventUnregister implements waiter.Waitable.EventUnregister. func (f *fileDescription) EventUnregister(e *waiter.Entry) { f.inode.queue.EventUnregister(e) - fdnotifier.UpdateFD(int32(f.inode.hostFD)) + if f.inode.mayBlock { + fdnotifier.UpdateFD(int32(f.inode.hostFD)) + } } // Readiness uses the poll() syscall to check the status of the underlying FD. diff --git a/pkg/sentry/fsimpl/host/host_state_autogen.go b/pkg/sentry/fsimpl/host/host_state_autogen.go index 3507e1aa1..6de6c002c 100644 --- a/pkg/sentry/fsimpl/host/host_state_autogen.go +++ b/pkg/sentry/fsimpl/host/host_state_autogen.go @@ -28,6 +28,77 @@ func (r *ConnectedEndpointRefs) StateLoad(stateSourceObject state.Source) { stateSourceObject.AfterLoad(r.afterLoad) } +func (i *inode) StateTypeName() string { + return "pkg/sentry/fsimpl/host.inode" +} + +func (i *inode) StateFields() []string { + return []string{ + "InodeNoStatFS", + "InodeNotDirectory", + "InodeNotSymlink", + "InodeTemporary", + "locks", + "inodeRefs", + "hostFD", + "ino", + "ftype", + "mayBlock", + "seekable", + "isTTY", + "savable", + "queue", + "mappings", + "pf", + "haveBuf", + "buf", + } +} + +func (i *inode) StateSave(stateSinkObject state.Sink) { + i.beforeSave() + stateSinkObject.Save(0, &i.InodeNoStatFS) + stateSinkObject.Save(1, &i.InodeNotDirectory) + stateSinkObject.Save(2, &i.InodeNotSymlink) + stateSinkObject.Save(3, &i.InodeTemporary) + stateSinkObject.Save(4, &i.locks) + stateSinkObject.Save(5, &i.inodeRefs) + stateSinkObject.Save(6, &i.hostFD) + stateSinkObject.Save(7, &i.ino) + stateSinkObject.Save(8, &i.ftype) + stateSinkObject.Save(9, &i.mayBlock) + stateSinkObject.Save(10, &i.seekable) + stateSinkObject.Save(11, &i.isTTY) + stateSinkObject.Save(12, &i.savable) + stateSinkObject.Save(13, &i.queue) + stateSinkObject.Save(14, &i.mappings) + stateSinkObject.Save(15, &i.pf) + stateSinkObject.Save(16, &i.haveBuf) + stateSinkObject.Save(17, &i.buf) +} + +func (i *inode) StateLoad(stateSourceObject state.Source) { + stateSourceObject.Load(0, &i.InodeNoStatFS) + stateSourceObject.Load(1, &i.InodeNotDirectory) + stateSourceObject.Load(2, &i.InodeNotSymlink) + stateSourceObject.Load(3, &i.InodeTemporary) + stateSourceObject.Load(4, &i.locks) + stateSourceObject.Load(5, &i.inodeRefs) + stateSourceObject.Load(6, &i.hostFD) + stateSourceObject.Load(7, &i.ino) + stateSourceObject.Load(8, &i.ftype) + stateSourceObject.Load(9, &i.mayBlock) + stateSourceObject.Load(10, &i.seekable) + stateSourceObject.Load(11, &i.isTTY) + stateSourceObject.Load(12, &i.savable) + stateSourceObject.Load(13, &i.queue) + stateSourceObject.Load(14, &i.mappings) + stateSourceObject.Load(15, &i.pf) + stateSourceObject.Load(16, &i.haveBuf) + stateSourceObject.Load(17, &i.buf) + stateSourceObject.AfterLoad(i.afterLoad) +} + func (f *filesystemType) StateTypeName() string { return "pkg/sentry/fsimpl/host.filesystemType" } @@ -73,71 +144,6 @@ func (fs *filesystem) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(1, &fs.devMinor) } -func (i *inode) StateTypeName() string { - return "pkg/sentry/fsimpl/host.inode" -} - -func (i *inode) StateFields() []string { - return []string{ - "InodeNoStatFS", - "InodeNotDirectory", - "InodeNotSymlink", - "InodeTemporary", - "locks", - "inodeRefs", - "hostFD", - "ino", - "isTTY", - "seekable", - "wouldBlock", - "queue", - "canMap", - "mappings", - "pf", - } -} - -func (i *inode) beforeSave() {} - -func (i *inode) StateSave(stateSinkObject state.Sink) { - i.beforeSave() - stateSinkObject.Save(0, &i.InodeNoStatFS) - stateSinkObject.Save(1, &i.InodeNotDirectory) - stateSinkObject.Save(2, &i.InodeNotSymlink) - stateSinkObject.Save(3, &i.InodeTemporary) - stateSinkObject.Save(4, &i.locks) - stateSinkObject.Save(5, &i.inodeRefs) - stateSinkObject.Save(6, &i.hostFD) - stateSinkObject.Save(7, &i.ino) - stateSinkObject.Save(8, &i.isTTY) - stateSinkObject.Save(9, &i.seekable) - stateSinkObject.Save(10, &i.wouldBlock) - stateSinkObject.Save(11, &i.queue) - stateSinkObject.Save(12, &i.canMap) - stateSinkObject.Save(13, &i.mappings) - stateSinkObject.Save(14, &i.pf) -} - -func (i *inode) afterLoad() {} - -func (i *inode) StateLoad(stateSourceObject state.Source) { - stateSourceObject.Load(0, &i.InodeNoStatFS) - stateSourceObject.Load(1, &i.InodeNotDirectory) - stateSourceObject.Load(2, &i.InodeNotSymlink) - stateSourceObject.Load(3, &i.InodeTemporary) - stateSourceObject.Load(4, &i.locks) - stateSourceObject.Load(5, &i.inodeRefs) - stateSourceObject.Load(6, &i.hostFD) - stateSourceObject.Load(7, &i.ino) - stateSourceObject.Load(8, &i.isTTY) - stateSourceObject.Load(9, &i.seekable) - stateSourceObject.Load(10, &i.wouldBlock) - stateSourceObject.Load(11, &i.queue) - stateSourceObject.Load(12, &i.canMap) - stateSourceObject.Load(13, &i.mappings) - stateSourceObject.Load(14, &i.pf) -} - func (f *fileDescription) StateTypeName() string { return "pkg/sentry/fsimpl/host.fileDescription" } @@ -216,12 +222,11 @@ func (i *inodePlatformFile) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(2, &i.fileMapper) } -func (i *inodePlatformFile) afterLoad() {} - func (i *inodePlatformFile) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &i.inode) stateSourceObject.Load(1, &i.fdRefs) stateSourceObject.Load(2, &i.fileMapper) + stateSourceObject.AfterLoad(i.afterLoad) } func (c *ConnectedEndpoint) StateTypeName() string { @@ -290,9 +295,9 @@ func (t *TTYFileDescription) StateLoad(stateSourceObject state.Source) { func init() { state.Register((*ConnectedEndpointRefs)(nil)) + state.Register((*inode)(nil)) state.Register((*filesystemType)(nil)) state.Register((*filesystem)(nil)) - state.Register((*inode)(nil)) state.Register((*fileDescription)(nil)) state.Register((*inodeRefs)(nil)) state.Register((*inodePlatformFile)(nil)) diff --git a/pkg/sentry/fsimpl/host/mmap.go b/pkg/sentry/fsimpl/host/mmap.go index b51a17bed..3d7eb2f96 100644 --- a/pkg/sentry/fsimpl/host/mmap.go +++ b/pkg/sentry/fsimpl/host/mmap.go @@ -43,7 +43,7 @@ type inodePlatformFile struct { fileMapper fsutil.HostFileMapper // fileMapperInitOnce is used to lazily initialize fileMapper. - fileMapperInitOnce sync.Once `state:"nosave"` // FIXME(gvisor.dev/issue/1663): not yet supported. + fileMapperInitOnce sync.Once `state:"nosave"` } // IncRef implements memmap.File.IncRef. diff --git a/pkg/sentry/fsimpl/host/save_restore.go b/pkg/sentry/fsimpl/host/save_restore.go new file mode 100644 index 000000000..7e32a8863 --- /dev/null +++ b/pkg/sentry/fsimpl/host/save_restore.go @@ -0,0 +1,78 @@ +// 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 host + +import ( + "fmt" + "io" + "sync/atomic" + "syscall" + + "gvisor.dev/gvisor/pkg/fdnotifier" + "gvisor.dev/gvisor/pkg/safemem" + "gvisor.dev/gvisor/pkg/sentry/hostfd" + "gvisor.dev/gvisor/pkg/usermem" +) + +// beforeSave is invoked by stateify. +func (i *inode) beforeSave() { + if !i.savable { + panic("host.inode is not savable") + } + if i.ftype == syscall.S_IFIFO { + // If this pipe FD is readable, drain it so that bytes in the pipe can + // be read after restore. (This is a legacy VFS1 feature.) We don't + // know if the pipe FD is readable, so just try reading and tolerate + // EBADF from the read. + i.bufMu.Lock() + defer i.bufMu.Unlock() + var buf [usermem.PageSize]byte + for { + n, err := hostfd.Preadv2(int32(i.hostFD), safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf[:])), -1 /* offset */, 0 /* flags */) + if n != 0 { + i.buf = append(i.buf, buf[:n]...) + } + if err != nil { + if err == io.EOF || err == syscall.EAGAIN || err == syscall.EBADF { + break + } + panic(fmt.Errorf("host.inode.beforeSave: buffering from pipe failed: %v", err)) + } + } + if len(i.buf) != 0 { + atomic.StoreUint32(&i.haveBuf, 1) + } + } +} + +// afterLoad is invoked by stateify. +func (i *inode) afterLoad() { + if i.mayBlock { + if err := syscall.SetNonblock(i.hostFD, true); err != nil { + panic(fmt.Sprintf("host.inode.afterLoad: failed to set host FD %d non-blocking: %v", i.hostFD, err)) + } + if err := fdnotifier.AddFD(int32(i.hostFD), &i.queue); err != nil { + panic(fmt.Sprintf("host.inode.afterLoad: fdnotifier.AddFD(%d) failed: %v", i.hostFD, err)) + } + } +} + +// afterLoad is invoked by stateify. +func (i *inodePlatformFile) afterLoad() { + if i.fileMapper.IsInited() { + // Ensure that we don't call i.fileMapper.Init() again. + i.fileMapperInitOnce.Do(func() {}) + } +} diff --git a/pkg/sentry/fsimpl/host/util.go b/pkg/sentry/fsimpl/host/util.go index 412bdb2eb..b2f43a119 100644 --- a/pkg/sentry/fsimpl/host/util.go +++ b/pkg/sentry/fsimpl/host/util.go @@ -43,12 +43,6 @@ func timespecToStatxTimestamp(ts unix.Timespec) linux.StatxTimestamp { return linux.StatxTimestamp{Sec: int64(ts.Sec), Nsec: uint32(ts.Nsec)} } -// wouldBlock returns true for file types that can return EWOULDBLOCK -// for blocking operations, e.g. pipes, character devices, and sockets. -func wouldBlock(fileType uint32) bool { - return fileType == syscall.S_IFIFO || fileType == syscall.S_IFCHR || fileType == syscall.S_IFSOCK -} - // isBlockError checks if an error is EAGAIN or EWOULDBLOCK. // If so, they can be transformed into syserror.ErrWouldBlock. func isBlockError(err error) bool { diff --git a/pkg/sentry/fsimpl/kernfs/fstree.go b/pkg/sentry/fsimpl/kernfs/fstree.go index ce86d7919..9dc52dabc 100644 --- a/pkg/sentry/fsimpl/kernfs/fstree.go +++ b/pkg/sentry/fsimpl/kernfs/fstree.go @@ -34,7 +34,7 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *Dentry, b if mnt == vfsroot.Mount() && &d.vfsd == vfsroot.Dentry() { return vfs.PrependPathAtVFSRootError{} } - if &d.vfsd == mnt.Root() { + if mnt != nil && &d.vfsd == mnt.Root() { return nil } if d.parent == nil { @@ -44,3 +44,12 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *Dentry, b d = d.parent } } + +// DebugPathname returns a pathname to d relative to its filesystem root. +// DebugPathname does not correspond to any Linux function; it's used to +// generate dentry pathnames for debugging. +func genericDebugPathname(d *Dentry) string { + var b fspath.Builder + _ = genericPrependPath(vfs.VirtualDentry{}, nil, d, &b) + return b.String() +} diff --git a/pkg/sentry/fsimpl/overlay/fstree.go b/pkg/sentry/fsimpl/overlay/fstree.go index 1fa1b4ad3..c3eb062ed 100644 --- a/pkg/sentry/fsimpl/overlay/fstree.go +++ b/pkg/sentry/fsimpl/overlay/fstree.go @@ -34,7 +34,7 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *dentry, b if mnt == vfsroot.Mount() && &d.vfsd == vfsroot.Dentry() { return vfs.PrependPathAtVFSRootError{} } - if &d.vfsd == mnt.Root() { + if mnt != nil && &d.vfsd == mnt.Root() { return nil } if d.parent == nil { @@ -44,3 +44,12 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *dentry, b d = d.parent } } + +// DebugPathname returns a pathname to d relative to its filesystem root. +// DebugPathname does not correspond to any Linux function; it's used to +// generate dentry pathnames for debugging. +func genericDebugPathname(d *dentry) string { + var b fspath.Builder + _ = genericPrependPath(vfs.VirtualDentry{}, nil, d, &b) + return b.String() +} diff --git a/pkg/sentry/fsimpl/overlay/overlay.go b/pkg/sentry/fsimpl/overlay/overlay.go index 4c5de8d32..f28411b5b 100644 --- a/pkg/sentry/fsimpl/overlay/overlay.go +++ b/pkg/sentry/fsimpl/overlay/overlay.go @@ -458,9 +458,9 @@ type dentry struct { // // - isMappable is non-zero iff wrappedMappable is non-nil. isMappable is // accessed using atomic memory operations. - mapsMu sync.Mutex + mapsMu sync.Mutex `state:"nosave"` lowerMappings memmap.MappingSet - dataMu sync.RWMutex + dataMu sync.RWMutex `state:"nosave"` wrappedMappable memmap.Mappable isMappable uint32 diff --git a/pkg/sentry/fsimpl/overlay/overlay_state_autogen.go b/pkg/sentry/fsimpl/overlay/overlay_state_autogen.go index f56cb8d02..bdab5d0be 100644 --- a/pkg/sentry/fsimpl/overlay/overlay_state_autogen.go +++ b/pkg/sentry/fsimpl/overlay/overlay_state_autogen.go @@ -147,9 +147,7 @@ func (d *dentry) StateFields() []string { "devMajor", "devMinor", "ino", - "mapsMu", "lowerMappings", - "dataMu", "wrappedMappable", "isMappable", "locks", @@ -178,13 +176,11 @@ func (d *dentry) StateSave(stateSinkObject state.Sink) { stateSinkObject.Save(14, &d.devMajor) stateSinkObject.Save(15, &d.devMinor) stateSinkObject.Save(16, &d.ino) - stateSinkObject.Save(17, &d.mapsMu) - stateSinkObject.Save(18, &d.lowerMappings) - stateSinkObject.Save(19, &d.dataMu) - stateSinkObject.Save(20, &d.wrappedMappable) - stateSinkObject.Save(21, &d.isMappable) - stateSinkObject.Save(22, &d.locks) - stateSinkObject.Save(23, &d.watches) + stateSinkObject.Save(17, &d.lowerMappings) + stateSinkObject.Save(18, &d.wrappedMappable) + stateSinkObject.Save(19, &d.isMappable) + stateSinkObject.Save(20, &d.locks) + stateSinkObject.Save(21, &d.watches) } func (d *dentry) afterLoad() {} @@ -207,13 +203,11 @@ func (d *dentry) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(14, &d.devMajor) stateSourceObject.Load(15, &d.devMinor) stateSourceObject.Load(16, &d.ino) - stateSourceObject.Load(17, &d.mapsMu) - stateSourceObject.Load(18, &d.lowerMappings) - stateSourceObject.Load(19, &d.dataMu) - stateSourceObject.Load(20, &d.wrappedMappable) - stateSourceObject.Load(21, &d.isMappable) - stateSourceObject.Load(22, &d.locks) - stateSourceObject.Load(23, &d.watches) + stateSourceObject.Load(17, &d.lowerMappings) + stateSourceObject.Load(18, &d.wrappedMappable) + stateSourceObject.Load(19, &d.isMappable) + stateSourceObject.Load(20, &d.locks) + stateSourceObject.Load(21, &d.watches) } func (fd *fileDescription) StateTypeName() string { diff --git a/pkg/sentry/fsimpl/tmpfs/fstree.go b/pkg/sentry/fsimpl/tmpfs/fstree.go index 2dd98a921..d46351488 100644 --- a/pkg/sentry/fsimpl/tmpfs/fstree.go +++ b/pkg/sentry/fsimpl/tmpfs/fstree.go @@ -34,7 +34,7 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *dentry, b if mnt == vfsroot.Mount() && &d.vfsd == vfsroot.Dentry() { return vfs.PrependPathAtVFSRootError{} } - if &d.vfsd == mnt.Root() { + if mnt != nil && &d.vfsd == mnt.Root() { return nil } if d.parent == nil { @@ -44,3 +44,12 @@ func genericPrependPath(vfsroot vfs.VirtualDentry, mnt *vfs.Mount, d *dentry, b d = d.parent } } + +// DebugPathname returns a pathname to d relative to its filesystem root. +// DebugPathname does not correspond to any Linux function; it's used to +// generate dentry pathnames for debugging. +func genericDebugPathname(d *dentry) string { + var b fspath.Builder + _ = genericPrependPath(vfs.VirtualDentry{}, nil, d, &b) + return b.String() +} diff --git a/pkg/sentry/fsimpl/tmpfs/regular_file.go b/pkg/sentry/fsimpl/tmpfs/regular_file.go index ce4e3eda7..98680fde9 100644 --- a/pkg/sentry/fsimpl/tmpfs/regular_file.go +++ b/pkg/sentry/fsimpl/tmpfs/regular_file.go @@ -42,7 +42,7 @@ type regularFile struct { inode inode // memFile is a platform.File used to allocate pages to this regularFile. - memFile *pgalloc.MemoryFile + memFile *pgalloc.MemoryFile `state:"nosave"` // memoryUsageKind is the memory accounting category under which pages backing // this regularFile's contents are accounted. @@ -92,7 +92,7 @@ type regularFile struct { func (fs *filesystem) newRegularFile(kuid auth.KUID, kgid auth.KGID, mode linux.FileMode) *inode { file := ®ularFile{ - memFile: fs.memFile, + memFile: fs.mfp.MemoryFile(), memoryUsageKind: usage.Tmpfs, seals: linux.F_SEAL_SEAL, } diff --git a/pkg/sentry/fsimpl/tmpfs/save_restore.go b/pkg/sentry/fsimpl/tmpfs/save_restore.go new file mode 100644 index 000000000..b27f75cc2 --- /dev/null +++ b/pkg/sentry/fsimpl/tmpfs/save_restore.go @@ -0,0 +1,20 @@ +// 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 tmpfs + +// afterLoad is called by stateify. +func (rf *regularFile) afterLoad() { + rf.memFile = rf.inode.fs.mfp.MemoryFile() +} diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go index e2a0aac69..4ce859d57 100644 --- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go +++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go @@ -61,8 +61,9 @@ type FilesystemType struct{} type filesystem struct { vfsfs vfs.Filesystem - // memFile is used to allocate pages to for regular files. - memFile *pgalloc.MemoryFile + // mfp is used to allocate memory that stores regular file contents. mfp is + // immutable. + mfp pgalloc.MemoryFileProvider // clock is a realtime clock used to set timestamps in file operations. clock time.Clock @@ -106,8 +107,8 @@ type FilesystemOpts struct { // GetFilesystem implements vfs.FilesystemType.GetFilesystem. func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, _ string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) { - memFileProvider := pgalloc.MemoryFileProviderFromContext(ctx) - if memFileProvider == nil { + mfp := pgalloc.MemoryFileProviderFromContext(ctx) + if mfp == nil { panic("MemoryFileProviderFromContext returned nil") } @@ -181,7 +182,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt } clock := time.RealtimeClockFromContext(ctx) fs := filesystem{ - memFile: memFileProvider.MemoryFile(), + mfp: mfp, clock: clock, devMinor: devMinor, } diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs_state_autogen.go b/pkg/sentry/fsimpl/tmpfs/tmpfs_state_autogen.go index 21681ba5c..21c53afaf 100644 --- a/pkg/sentry/fsimpl/tmpfs/tmpfs_state_autogen.go +++ b/pkg/sentry/fsimpl/tmpfs/tmpfs_state_autogen.go @@ -212,7 +212,6 @@ func (rf *regularFile) StateTypeName() string { func (rf *regularFile) StateFields() []string { return []string{ "inode", - "memFile", "memoryUsageKind", "mappings", "writableMappingPages", @@ -227,26 +226,23 @@ func (rf *regularFile) beforeSave() {} func (rf *regularFile) StateSave(stateSinkObject state.Sink) { rf.beforeSave() stateSinkObject.Save(0, &rf.inode) - stateSinkObject.Save(1, &rf.memFile) - stateSinkObject.Save(2, &rf.memoryUsageKind) - stateSinkObject.Save(3, &rf.mappings) - stateSinkObject.Save(4, &rf.writableMappingPages) - stateSinkObject.Save(5, &rf.data) - stateSinkObject.Save(6, &rf.seals) - stateSinkObject.Save(7, &rf.size) + stateSinkObject.Save(1, &rf.memoryUsageKind) + stateSinkObject.Save(2, &rf.mappings) + stateSinkObject.Save(3, &rf.writableMappingPages) + stateSinkObject.Save(4, &rf.data) + stateSinkObject.Save(5, &rf.seals) + stateSinkObject.Save(6, &rf.size) } -func (rf *regularFile) afterLoad() {} - func (rf *regularFile) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &rf.inode) - stateSourceObject.Load(1, &rf.memFile) - stateSourceObject.Load(2, &rf.memoryUsageKind) - stateSourceObject.Load(3, &rf.mappings) - stateSourceObject.Load(4, &rf.writableMappingPages) - stateSourceObject.Load(5, &rf.data) - stateSourceObject.Load(6, &rf.seals) - stateSourceObject.Load(7, &rf.size) + stateSourceObject.Load(1, &rf.memoryUsageKind) + stateSourceObject.Load(2, &rf.mappings) + stateSourceObject.Load(3, &rf.writableMappingPages) + stateSourceObject.Load(4, &rf.data) + stateSourceObject.Load(5, &rf.seals) + stateSourceObject.Load(6, &rf.size) + stateSourceObject.AfterLoad(rf.afterLoad) } func (fd *regularFileFD) StateTypeName() string { @@ -353,7 +349,7 @@ func (fs *filesystem) StateTypeName() string { func (fs *filesystem) StateFields() []string { return []string{ "vfsfs", - "memFile", + "mfp", "clock", "devMinor", "nextInoMinusOne", @@ -366,7 +362,7 @@ func (fs *filesystem) beforeSave() {} func (fs *filesystem) StateSave(stateSinkObject state.Sink) { fs.beforeSave() stateSinkObject.Save(0, &fs.vfsfs) - stateSinkObject.Save(1, &fs.memFile) + stateSinkObject.Save(1, &fs.mfp) stateSinkObject.Save(2, &fs.clock) stateSinkObject.Save(3, &fs.devMinor) stateSinkObject.Save(4, &fs.nextInoMinusOne) @@ -377,7 +373,7 @@ func (fs *filesystem) afterLoad() {} func (fs *filesystem) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(0, &fs.vfsfs) - stateSourceObject.Load(1, &fs.memFile) + stateSourceObject.Load(1, &fs.mfp) stateSourceObject.Load(2, &fs.clock) stateSourceObject.Load(3, &fs.devMinor) stateSourceObject.Load(4, &fs.nextInoMinusOne) |