summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl/gofer
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2020-10-24 00:53:27 +0000
committergVisor bot <gvisor-bot@google.com>2020-10-24 00:53:27 +0000
commit70cbe923d390e91fc05b06dfa05c6810b0701037 (patch)
treefba2e949690078ba4ffef65d03a2bdfaa9b01b61 /pkg/sentry/fsimpl/gofer
parentb1dbae4ae486840d512fc3e0ba8606ae66580234 (diff)
parent9f87400f087df0492cf181c97f431b6d5ce3a987 (diff)
Merge release-20201019.0-51-g9f87400f0 (automated)
Diffstat (limited to 'pkg/sentry/fsimpl/gofer')
-rw-r--r--pkg/sentry/fsimpl/gofer/directory.go6
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go29
-rw-r--r--pkg/sentry/fsimpl/gofer/fstree.go11
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go325
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer_state_autogen.go189
-rw-r--r--pkg/sentry/fsimpl/gofer/host_named_pipe.go20
-rw-r--r--pkg/sentry/fsimpl/gofer/regular_file.go22
-rw-r--r--pkg/sentry/fsimpl/gofer/save_restore.go327
-rw-r--r--pkg/sentry/fsimpl/gofer/socket.go7
-rw-r--r--pkg/sentry/fsimpl/gofer/special_file.go108
10 files changed, 763 insertions, 281 deletions
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
}