summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/sentry/fsimpl/gofer/BUILD1
-rw-r--r--pkg/sentry/fsimpl/gofer/directory.go12
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go18
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go308
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer_test.go4
-rw-r--r--pkg/sentry/fsimpl/gofer/handle.go19
-rw-r--r--pkg/sentry/fsimpl/gofer/regular_file.go118
-rw-r--r--pkg/sentry/fsimpl/gofer/special_file.go11
-rw-r--r--pkg/sentry/vfs/file_description.go5
9 files changed, 311 insertions, 185 deletions
diff --git a/pkg/sentry/fsimpl/gofer/BUILD b/pkg/sentry/fsimpl/gofer/BUILD
index 4a800dcf9..16787116f 100644
--- a/pkg/sentry/fsimpl/gofer/BUILD
+++ b/pkg/sentry/fsimpl/gofer/BUILD
@@ -85,5 +85,6 @@ go_test(
deps = [
"//pkg/p9",
"//pkg/sentry/contexttest",
+ "//pkg/sentry/pgalloc",
],
)
diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go
index 1679066ba..2a8011eb4 100644
--- a/pkg/sentry/fsimpl/gofer/directory.go
+++ b/pkg/sentry/fsimpl/gofer/directory.go
@@ -90,10 +90,8 @@ func (d *dentry) createSyntheticChildLocked(opts *createSyntheticOpts) {
uid: uint32(opts.kuid),
gid: uint32(opts.kgid),
blockSize: usermem.PageSize, // arbitrary
- handle: handle{
- fd: -1,
- },
- nlink: uint32(2),
+ hostFD: -1,
+ nlink: uint32(2),
}
switch opts.mode.FileType() {
case linux.S_IFDIR:
@@ -205,14 +203,14 @@ func (d *dentry) getDirents(ctx context.Context) ([]vfs.Dirent, error) {
off := uint64(0)
const count = 64 * 1024 // for consistency with the vfs1 client
d.handleMu.RLock()
- if !d.handleReadable {
+ if d.readFile.isNil() {
// This should not be possible because a readable handle should
// have been opened when the calling directoryFD was opened.
d.handleMu.RUnlock()
panic("gofer.dentry.getDirents called without a readable handle")
}
for {
- p9ds, err := d.handle.file.readdir(ctx, off, count)
+ p9ds, err := d.readFile.readdir(ctx, off, count)
if err != nil {
d.handleMu.RUnlock()
return nil, err
@@ -304,5 +302,5 @@ func (fd *directoryFD) Seek(ctx context.Context, offset int64, whence int32) (in
// Sync implements vfs.FileDescriptionImpl.Sync.
func (fd *directoryFD) Sync(ctx context.Context) error {
- return fd.dentry().handle.sync(ctx)
+ return fd.dentry().syncRemoteFile(ctx)
}
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 582b744bb..eaef2594d 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -55,7 +55,7 @@ func (fs *filesystem) Sync(ctx context.Context) error {
// Sync regular files.
for _, d := range ds {
- err := d.syncSharedHandle(ctx)
+ err := d.syncCachedFile(ctx)
d.DecRef(ctx)
if err != nil && retErr == nil {
retErr = err
@@ -1107,12 +1107,18 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
useRegularFileFD := child.fileType() == linux.S_IFREG && !d.fs.opts.regularFilesUseSpecialFileFD
if useRegularFileFD {
child.handleMu.Lock()
- child.handle.file = openFile
- if fdobj != nil {
- child.handle.fd = int32(fdobj.Release())
+ if vfs.MayReadFileWithOpenFlags(opts.Flags) {
+ child.readFile = openFile
+ if fdobj != nil {
+ child.hostFD = int32(fdobj.Release())
+ }
+ } else if fdobj != nil {
+ // Can't use fdobj if it's not readable.
+ fdobj.Close()
+ }
+ if vfs.MayWriteFileWithOpenFlags(opts.Flags) {
+ child.writeFile = openFile
}
- child.handleReadable = vfs.MayReadFileWithOpenFlags(opts.Flags)
- child.handleWritable = vfs.MayWriteFileWithOpenFlags(opts.Flags)
child.handleMu.Unlock()
}
// Insert the dentry into the tree.
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index 59323086d..f1d3bf911 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -506,9 +506,9 @@ func (fs *filesystem) Release(ctx context.Context) {
for d := range fs.syncableDentries {
d.handleMu.Lock()
d.dataMu.Lock()
- if d.handleWritable {
+ if h := d.writeHandleLocked(); h.isOpen() {
// Write dirty cached data to the remote file.
- if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, fs.mfp.MemoryFile(), d.handle.writeFromBlocksAt); err != nil {
+ if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, fs.mfp.MemoryFile(), h.writeFromBlocksAt); err != nil {
log.Warningf("gofer.filesystem.Release: failed to flush dentry: %v", err)
}
// TODO(jamieliu): Do we need to flushf/fsync d?
@@ -518,9 +518,9 @@ func (fs *filesystem) Release(ctx context.Context) {
d.dirty.RemoveAll()
d.dataMu.Unlock()
// Close the host fd if one exists.
- if d.handle.fd >= 0 {
- syscall.Close(int(d.handle.fd))
- d.handle.fd = -1
+ if d.hostFD >= 0 {
+ syscall.Close(int(d.hostFD))
+ d.hostFD = -1
}
d.handleMu.Unlock()
}
@@ -622,7 +622,12 @@ type dentry struct {
mtime int64
ctime int64
btime int64
- // File size, protected by both metadataMu and dataMu (i.e. both must be
+ // File size, which differs from other metadata in two ways:
+ //
+ // - We make a best-effort attempt to keep it up to date even if
+ // !dentry.cachedMetadataAuthoritative() for the sake of O_APPEND writes.
+ //
+ // - size is protected by both metadataMu and dataMu (i.e. both must be
// locked to mutate it; locking either is sufficient to access it).
size uint64
// If this dentry does not represent a synthetic file, deleted is 0, and
@@ -643,30 +648,28 @@ type dentry struct {
// the file into memmap.MappingSpaces. mappings is protected by mapsMu.
mappings memmap.MappingSet
- // If this dentry represents a regular file or directory:
- //
- // - handle is the I/O handle used by all regularFileFDs/directoryFDs
- // representing this dentry.
- //
- // - handleReadable is true if handle is readable.
- //
- // - handleWritable is true if handle is writable.
- //
- // Invariants:
+ // - If this dentry represents a regular file or directory, readFile is the
+ // p9.File used for reads by all regularFileFDs/directoryFDs representing
+ // this dentry.
//
- // - If handleReadable == handleWritable == false, then handle.file == nil
- // (i.e. there is no open handle). Conversely, if handleReadable ||
- // handleWritable == true, then handle.file != nil (i.e. there is an open
- // handle).
+ // - If this dentry represents a regular file, writeFile is the p9.File
+ // used for writes by all regularFileFDs representing this dentry.
//
- // - handleReadable and handleWritable cannot transition from true to false
- // (i.e. handles may not be downgraded).
+ // - If this dentry represents a regular file, hostFD is the host FD used
+ // for memory mappings and I/O (when applicable) in preference to readFile
+ // and writeFile. hostFD is always readable; if !writeFile.isNil(), it must
+ // also be writable. If hostFD is -1, no such host FD is available.
//
// These fields are protected by handleMu.
- handleMu sync.RWMutex
- handle handle
- handleReadable bool
- handleWritable bool
+ //
+ // readFile and writeFile may or may not represent the same p9.File. Once
+ // either p9.File transitions from closed (isNil() == true) to open
+ // (isNil() == false), it may be mutated with handleMu locked, but cannot
+ // be closed until the dentry is destroyed.
+ handleMu sync.RWMutex
+ readFile p9file
+ writeFile p9file
+ hostFD int32
dataMu sync.RWMutex
@@ -680,7 +683,7 @@ type dentry struct {
// tracks dirty segments in cache. dirty is protected by dataMu.
dirty fsutil.DirtySet
- // pf implements platform.File for mappings of handle.fd.
+ // pf implements platform.File for mappings of hostFD.
pf dentryPlatformFile
// If this dentry represents a symbolic link, InteropModeShared is not in
@@ -742,9 +745,7 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma
uid: uint32(fs.opts.dfltuid),
gid: uint32(fs.opts.dfltgid),
blockSize: usermem.PageSize,
- handle: handle{
- fd: -1,
- },
+ hostFD: -1,
}
d.pf.dentry = d
if mask.UID {
@@ -835,9 +836,13 @@ func (d *dentry) updateFromP9AttrsLocked(mask p9.AttrMask, attr *p9.Attr) {
// Preconditions: !d.isSynthetic()
func (d *dentry) updateFromGetattr(ctx context.Context) error {
- // Use d.handle.file, which represents a 9P fid that has been opened, in
- // preference to d.file, which represents a 9P fid that has not. This may
- // be significantly more efficient in some implementations.
+ // 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.
+ // This may be significantly more efficient in some implementations. Prefer
+ // d.writeFile over d.readFile since some filesystem implementations may
+ // update a writable handle's metadata after writes to that handle, without
+ // making metadata updates immediately visible to read-only handles
+ // representing the same file.
var (
file p9file
handleMuRLocked bool
@@ -847,8 +852,11 @@ func (d *dentry) updateFromGetattr(ctx context.Context) error {
d.metadataMu.Lock()
defer d.metadataMu.Unlock()
d.handleMu.RLock()
- if !d.handle.file.isNil() {
- file = d.handle.file
+ if !d.writeFile.isNil() {
+ file = d.writeFile
+ handleMuRLocked = true
+ } else if !d.readFile.isNil() {
+ file = d.readFile
handleMuRLocked = true
} else {
file = d.file
@@ -973,8 +981,9 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
return err
}
if stat.Mask&linux.STATX_SIZE != 0 {
- // Privatized copy-on-write mappings of truncated pages need to
- // be invalidated even if InteropModeShared is in effect.
+ // d.size should be kept up to date, and privatized
+ // copy-on-write mappings of truncated pages need to be
+ // invalidated, even if InteropModeShared is in effect.
d.updateFileSizeLocked(stat.Size)
}
}
@@ -1245,22 +1254,31 @@ func (d *dentry) destroyLocked(ctx context.Context) {
panic("dentry.destroyLocked() called with references on the dentry")
}
+ mf := d.fs.mfp.MemoryFile()
d.handleMu.Lock()
- if !d.handle.file.isNil() {
- mf := d.fs.mfp.MemoryFile()
- d.dataMu.Lock()
+ d.dataMu.Lock()
+ if h := d.writeHandleLocked(); h.isOpen() {
// Write dirty pages back to the remote filesystem.
- if d.handleWritable {
- if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, mf, d.handle.writeFromBlocksAt); err != nil {
- log.Warningf("gofer.dentry.destroyLocked: failed to write dirty data back: %v", err)
- }
+ if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, mf, h.writeFromBlocksAt); err != nil {
+ log.Warningf("gofer.dentry.destroyLocked: failed to write dirty data back: %v", err)
}
- // Discard cached data.
- d.cache.DropAll(mf)
- d.dirty.RemoveAll()
- d.dataMu.Unlock()
- // Clunk open fids and close open host FDs.
- d.handle.close(ctx)
+ }
+ // Discard cached data.
+ d.cache.DropAll(mf)
+ d.dirty.RemoveAll()
+ d.dataMu.Unlock()
+ // Clunk open fids and close open host FDs.
+ if !d.readFile.isNil() {
+ d.readFile.close(ctx)
+ }
+ if !d.writeFile.isNil() && d.readFile != d.writeFile {
+ d.writeFile.close(ctx)
+ }
+ d.readFile = p9file{}
+ d.writeFile = p9file{}
+ if d.hostFD >= 0 {
+ syscall.Close(int(d.hostFD))
+ d.hostFD = -1
}
d.handleMu.Unlock()
@@ -1393,80 +1411,120 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
// O_TRUNC).
if !trunc {
d.handleMu.RLock()
- if (!read || d.handleReadable) && (!write || d.handleWritable) {
- // The current handle is sufficient.
+ if (!read || !d.readFile.isNil()) && (!write || !d.writeFile.isNil()) {
+ // Current handles are sufficient.
d.handleMu.RUnlock()
return nil
}
d.handleMu.RUnlock()
}
- haveOldFD := false
+ fdToClose := int32(-1)
+ invalidateTranslations := false
d.handleMu.Lock()
- if (read && !d.handleReadable) || (write && !d.handleWritable) || trunc {
- // Get a new handle.
- wantReadable := d.handleReadable || read
- wantWritable := d.handleWritable || write
- h, err := openHandle(ctx, d.file, wantReadable, wantWritable, trunc)
+ if (read && d.readFile.isNil()) || (write && d.writeFile.isNil()) || trunc {
+ // Get a new handle. If this file has been opened for both reading and
+ // writing, try to get a single handle that is usable for both:
+ //
+ // - Writable memory mappings of a host FD require that the host FD is
+ // opened for both reading and writing.
+ //
+ // - NOTE(b/141991141): Some filesystems may not ensure coherence
+ // between multiple handles for the same file.
+ openReadable := !d.readFile.isNil() || read
+ openWritable := !d.writeFile.isNil() || write
+ h, err := openHandle(ctx, d.file, openReadable, openWritable, trunc)
+ if err == syserror.EACCES && (openReadable != read || openWritable != write) {
+ // It may not be possible to use a single handle for both
+ // reading and writing, since permissions on the file may have
+ // changed to e.g. disallow reading after previously being
+ // opened for reading. In this case, we have no choice but to
+ // use separate handles for reading and writing.
+ ctx.Debugf("gofer.dentry.ensureSharedHandle: bifurcating read/write handles for dentry %p", d)
+ openReadable = read
+ openWritable = write
+ h, err = openHandle(ctx, d.file, openReadable, openWritable, trunc)
+ }
if err != nil {
d.handleMu.Unlock()
return err
}
- if !d.handle.file.isNil() {
- // Check that old and new handles are compatible: If the old handle
- // includes a host file descriptor but the new one does not, or
- // vice versa, old and new memory mappings may be incoherent.
- haveOldFD = d.handle.fd >= 0
- haveNewFD := h.fd >= 0
- if haveOldFD != haveNewFD {
- d.handleMu.Unlock()
- ctx.Warningf("gofer.dentry.ensureSharedHandle: can't change host FD availability from %v to %v across dentry handle upgrade", haveOldFD, haveNewFD)
- h.close(ctx)
- return syserror.EIO
- }
- if haveOldFD {
- // We may have raced with callers of d.pf.FD() that are now
- // using the old file descriptor, preventing us from safely
- // closing it. We could handle this by invalidating existing
- // memmap.Translations, but this is expensive. Instead, use
- // dup3 to make the old file descriptor refer to the new file
- // description, then close the new file descriptor (which is no
- // longer needed). Racing callers may use the old or new file
- // description, but this doesn't matter since they refer to the
- // same file (unless d.fs.opts.overlayfsStaleRead is true,
- // which we handle separately).
- if err := syscall.Dup3(int(h.fd), int(d.handle.fd), syscall.O_CLOEXEC); err != nil {
+
+ if d.hostFD < 0 && openReadable && h.fd >= 0 {
+ // We have no existing FD; use the new FD for at least reading.
+ d.hostFD = h.fd
+ } else if d.hostFD >= 0 && d.writeFile.isNil() && openWritable {
+ // We have an existing read-only FD, but the file has just been
+ // opened for writing, so we need to start supporting writable memory
+ // mappings. This may race with callers of d.pf.FD() using the existing
+ // FD, so in most cases we need to delay closing the old FD until after
+ // invalidating memmap.Translations that might have observed it.
+ if !openReadable || h.fd < 0 {
+ // We don't have a read/write FD, so we have no FD that can be
+ // used to create writable memory mappings. Switch to using the
+ // internal page cache.
+ invalidateTranslations = true
+ fdToClose = d.hostFD
+ d.hostFD = -1
+ } else if d.fs.opts.overlayfsStaleRead {
+ // We do have a read/write FD, but it may not be coherent with
+ // the existing read-only FD, so we must switch to mappings of
+ // the new FD in both the application and sentry.
+ if err := d.pf.hostFileMapper.RegenerateMappings(int(h.fd)); err != nil {
d.handleMu.Unlock()
- ctx.Warningf("gofer.dentry.ensureSharedHandle: failed to dup fd %d to fd %d: %v", h.fd, d.handle.fd, err)
+ ctx.Warningf("gofer.dentry.ensureSharedHandle: failed to replace sentry mappings of old FD with mappings of new FD: %v", err)
h.close(ctx)
return err
}
- syscall.Close(int(h.fd))
- h.fd = d.handle.fd
- if d.fs.opts.overlayfsStaleRead {
- // Replace sentry mappings of the old FD with mappings of
- // the new FD, since the two are not necessarily coherent.
- if err := d.pf.hostFileMapper.RegenerateMappings(int(h.fd)); err != nil {
- d.handleMu.Unlock()
- ctx.Warningf("gofer.dentry.ensureSharedHandle: failed to replace sentry mappings of old FD with mappings of new FD: %v", err)
- h.close(ctx)
- return err
- }
+ invalidateTranslations = true
+ fdToClose = d.hostFD
+ d.hostFD = h.fd
+ } else {
+ // We do have a read/write FD. To avoid invalidating existing
+ // memmap.Translations (which is expensive), use dup3 to make
+ // the old file descriptor refer to the new file description,
+ // then close the new file descriptor (which is no longer
+ // needed). Racing callers of d.pf.FD() may use the old or new
+ // file description, but this doesn't matter since they refer
+ // to the same file, and any racing mappings must be read-only.
+ if err := syscall.Dup3(int(h.fd), int(d.hostFD), syscall.O_CLOEXEC); err != nil {
+ oldHostFD := d.hostFD
+ d.handleMu.Unlock()
+ ctx.Warningf("gofer.dentry.ensureSharedHandle: failed to dup fd %d to fd %d: %v", h.fd, oldHostFD, err)
+ h.close(ctx)
+ return err
}
- // Clunk the old fid before making the new handle visible (by
- // unlocking d.handleMu).
- d.handle.file.close(ctx)
+ fdToClose = h.fd
}
+ } else {
+ // h.fd is not useful.
+ fdToClose = h.fd
+ }
+
+ // Switch to new fids.
+ var oldReadFile p9file
+ if openReadable {
+ oldReadFile = d.readFile
+ d.readFile = h.file
+ }
+ var oldWriteFile p9file
+ if openWritable {
+ oldWriteFile = d.writeFile
+ d.writeFile = h.file
+ }
+ // NOTE(b/141991141): Clunk old fids before making new fids visible (by
+ // unlocking d.handleMu).
+ if !oldReadFile.isNil() {
+ oldReadFile.close(ctx)
+ }
+ if !oldWriteFile.isNil() && oldReadFile != oldWriteFile {
+ oldWriteFile.close(ctx)
}
- // Switch to the new handle.
- d.handle = h
- d.handleReadable = wantReadable
- d.handleWritable = wantWritable
}
d.handleMu.Unlock()
- if d.fs.opts.overlayfsStaleRead && haveOldFD {
- // Invalidate application mappings that may be using the old FD; they
+ if invalidateTranslations {
+ // Invalidate application mappings that may be using an old FD; they
// will be replaced with mappings using the new FD after future calls
// to d.Translate(). This requires holding d.mapsMu, which precedes
// d.handleMu in the lock order.
@@ -1474,7 +1532,51 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
d.mappings.InvalidateAll(memmap.InvalidateOpts{})
d.mapsMu.Unlock()
}
+ if fdToClose >= 0 {
+ syscall.Close(int(fdToClose))
+ }
+
+ return nil
+}
+// Preconditions: d.handleMu must be locked.
+func (d *dentry) readHandleLocked() handle {
+ return handle{
+ file: d.readFile,
+ fd: d.hostFD,
+ }
+}
+
+// Preconditions: d.handleMu must be locked.
+func (d *dentry) writeHandleLocked() handle {
+ return handle{
+ file: d.writeFile,
+ fd: d.hostFD,
+ }
+}
+
+func (d *dentry) syncRemoteFile(ctx context.Context) error {
+ d.handleMu.RLock()
+ defer d.handleMu.RUnlock()
+ return d.syncRemoteFileLocked(ctx)
+}
+
+// Preconditions: d.handleMu must be locked.
+func (d *dentry) syncRemoteFileLocked(ctx context.Context) error {
+ // If we have a host FD, fsyncing it is likely to be faster than an fsync
+ // RPC.
+ if d.hostFD >= 0 {
+ ctx.UninterruptibleSleepStart(false)
+ err := syscall.Fsync(int(d.hostFD))
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+ }
+ if !d.writeFile.isNil() {
+ return d.writeFile.fsync(ctx)
+ }
+ if !d.readFile.isNil() {
+ return d.readFile.fsync(ctx)
+ }
return nil
}
diff --git a/pkg/sentry/fsimpl/gofer/gofer_test.go b/pkg/sentry/fsimpl/gofer/gofer_test.go
index 56d80bcf8..36cca3625 100644
--- a/pkg/sentry/fsimpl/gofer/gofer_test.go
+++ b/pkg/sentry/fsimpl/gofer/gofer_test.go
@@ -20,10 +20,13 @@ import (
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
+ "gvisor.dev/gvisor/pkg/sentry/pgalloc"
)
func TestDestroyIdempotent(t *testing.T) {
+ ctx := contexttest.Context(t)
fs := filesystem{
+ mfp: pgalloc.MemoryFileProviderFromContext(ctx),
syncableDentries: make(map[*dentry]struct{}),
opts: filesystemOptions{
// Test relies on no dentry being held in the cache.
@@ -31,7 +34,6 @@ func TestDestroyIdempotent(t *testing.T) {
},
}
- ctx := contexttest.Context(t)
attr := &p9.Attr{
Mode: p9.ModeRegular,
}
diff --git a/pkg/sentry/fsimpl/gofer/handle.go b/pkg/sentry/fsimpl/gofer/handle.go
index 8792ca4f2..104157512 100644
--- a/pkg/sentry/fsimpl/gofer/handle.go
+++ b/pkg/sentry/fsimpl/gofer/handle.go
@@ -63,6 +63,10 @@ func openHandle(ctx context.Context, file p9file, read, write, trunc bool) (hand
}, nil
}
+func (h *handle) isOpen() bool {
+ return !h.file.isNil()
+}
+
func (h *handle) close(ctx context.Context) {
h.file.close(ctx)
h.file = p9file{}
@@ -124,18 +128,3 @@ func (h *handle) writeFromBlocksAt(ctx context.Context, srcs safemem.BlockSeq, o
}
return cp, cperr
}
-
-func (h *handle) sync(ctx context.Context) error {
- // Handle most common case first.
- if h.fd >= 0 {
- ctx.UninterruptibleSleepStart(false)
- err := syscall.Fsync(int(h.fd))
- ctx.UninterruptibleSleepFinish(false)
- return err
- }
- if h.file.isNil() {
- // File hasn't been touched, there is nothing to sync.
- return nil
- }
- return h.file.fsync(ctx)
-}
diff --git a/pkg/sentry/fsimpl/gofer/regular_file.go b/pkg/sentry/fsimpl/gofer/regular_file.go
index db6bed4f6..7e1cbf065 100644
--- a/pkg/sentry/fsimpl/gofer/regular_file.go
+++ b/pkg/sentry/fsimpl/gofer/regular_file.go
@@ -64,34 +64,34 @@ func (fd *regularFileFD) OnClose(ctx context.Context) error {
}
d.handleMu.RLock()
defer d.handleMu.RUnlock()
- return d.handle.file.flush(ctx)
+ if d.writeFile.isNil() {
+ return nil
+ }
+ return d.writeFile.flush(ctx)
}
// Allocate implements vfs.FileDescriptionImpl.Allocate.
func (fd *regularFileFD) Allocate(ctx context.Context, mode, offset, length uint64) error {
-
d := fd.dentry()
d.metadataMu.Lock()
defer d.metadataMu.Unlock()
- size := offset + length
-
// Allocating a smaller size is a noop.
- if size <= d.size {
+ size := offset + length
+ if d.cachedMetadataAuthoritative() && size <= d.size {
return nil
}
- d.handleMu.Lock()
- defer d.handleMu.Unlock()
-
- err := d.handle.file.allocate(ctx, p9.ToAllocateMode(mode), offset, length)
+ d.handleMu.RLock()
+ err := d.writeFile.allocate(ctx, p9.ToAllocateMode(mode), offset, length)
+ d.handleMu.RUnlock()
if err != nil {
return err
}
d.dataMu.Lock()
atomic.StoreUint64(&d.size, size)
d.dataMu.Unlock()
- if !d.cachedMetadataAuthoritative() {
+ if d.cachedMetadataAuthoritative() {
d.touchCMtimeLocked()
}
return nil
@@ -113,7 +113,7 @@ func (fd *regularFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs
// Check for reading at EOF before calling into MM (but not under
// InteropModeShared, which makes d.size unreliable).
d := fd.dentry()
- if d.fs.opts.interop != InteropModeShared && uint64(offset) >= atomic.LoadUint64(&d.size) {
+ if d.cachedMetadataAuthoritative() && uint64(offset) >= atomic.LoadUint64(&d.size) {
return 0, io.EOF
}
@@ -217,16 +217,23 @@ func (fd *regularFileFD) pwrite(ctx context.Context, src usermem.IOSequence, off
n, err := src.CopyInTo(ctx, rw)
if err != nil {
- return n, offset, err
+ return n, offset + n, err
}
if n > 0 && fd.vfsfd.StatusFlags()&(linux.O_DSYNC|linux.O_SYNC) != 0 {
- // Write dirty cached pages touched by the write back to the remote file.
+ // Note that if any of the following fail, then we can't guarantee that
+ // any data was actually written with the semantics of O_DSYNC or
+ // O_SYNC, so we return zero bytes written. Compare Linux's
+ // mm/filemap.c:generic_file_write_iter() =>
+ // include/linux/fs.h:generic_write_sync().
+ //
+ // Write dirty cached pages touched by the write back to the remote
+ // file.
if err := d.writeback(ctx, offset, src.NumBytes()); err != nil {
- return n, offset, err
+ return 0, offset, err
}
// Request the remote filesystem to sync the remote file.
- if err := d.handle.sync(ctx); err != nil {
- return n, offset, err
+ if err := d.syncRemoteFile(ctx); err != nil {
+ return 0, offset, err
}
}
return n, offset + n, nil
@@ -317,10 +324,11 @@ func (rw *dentryReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error)
// coherence with memory-mapped I/O), or if InteropModeShared is in effect
// (which prevents us from caching file contents and makes dentry.size
// unreliable), or if the file was opened O_DIRECT, read directly from
- // dentry.handle without locking dentry.dataMu.
+ // dentry.readHandleLocked() without locking dentry.dataMu.
rw.d.handleMu.RLock()
- if (rw.d.handle.fd >= 0 && !rw.d.fs.opts.forcePageCache) || rw.d.fs.opts.interop == InteropModeShared || rw.direct {
- n, err := rw.d.handle.readToBlocksAt(rw.ctx, dsts, rw.off)
+ h := rw.d.readHandleLocked()
+ if (rw.d.hostFD >= 0 && !rw.d.fs.opts.forcePageCache) || rw.d.fs.opts.interop == InteropModeShared || rw.direct {
+ n, err := h.readToBlocksAt(rw.ctx, dsts, rw.off)
rw.d.handleMu.RUnlock()
rw.off += n
return n, err
@@ -388,7 +396,7 @@ func (rw *dentryReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error)
End: gapEnd,
}
optMR := gap.Range()
- err := rw.d.cache.Fill(rw.ctx, reqMR, maxFillRange(reqMR, optMR), mf, usage.PageCache, rw.d.handle.readToBlocksAt)
+ err := rw.d.cache.Fill(rw.ctx, reqMR, maxFillRange(reqMR, optMR), mf, usage.PageCache, h.readToBlocksAt)
mf.MarkEvictable(rw.d, pgalloc.EvictableRange{optMR.Start, optMR.End})
seg, gap = rw.d.cache.Find(rw.off)
if !seg.Ok() {
@@ -403,7 +411,7 @@ func (rw *dentryReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error)
} else {
// Read directly from the file.
gapDsts := dsts.TakeFirst64(gapMR.Length())
- n, err := rw.d.handle.readToBlocksAt(rw.ctx, gapDsts, gapMR.Start)
+ n, err := h.readToBlocksAt(rw.ctx, gapDsts, gapMR.Start)
done += n
rw.off += n
dsts = dsts.DropFirst64(n)
@@ -435,11 +443,12 @@ func (rw *dentryReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, erro
// If we have a mmappable host FD (which must be used here to ensure
// coherence with memory-mapped I/O), or if InteropModeShared is in effect
// (which prevents us from caching file contents), or if the file was
- // opened with O_DIRECT, write directly to dentry.handle without locking
- // dentry.dataMu.
+ // opened with O_DIRECT, write directly to dentry.writeHandleLocked()
+ // without locking dentry.dataMu.
rw.d.handleMu.RLock()
- if (rw.d.handle.fd >= 0 && !rw.d.fs.opts.forcePageCache) || rw.d.fs.opts.interop == InteropModeShared || rw.direct {
- n, err := rw.d.handle.writeFromBlocksAt(rw.ctx, srcs, rw.off)
+ h := rw.d.writeHandleLocked()
+ if (rw.d.hostFD >= 0 && !rw.d.fs.opts.forcePageCache) || rw.d.fs.opts.interop == InteropModeShared || rw.direct {
+ n, err := h.writeFromBlocksAt(rw.ctx, srcs, rw.off)
rw.off += n
rw.d.dataMu.Lock()
if rw.off > rw.d.size {
@@ -501,7 +510,7 @@ func (rw *dentryReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, erro
// for detecting or avoiding this.
gapMR := gap.Range().Intersect(mr)
gapSrcs := srcs.TakeFirst64(gapMR.Length())
- n, err := rw.d.handle.writeFromBlocksAt(rw.ctx, gapSrcs, gapMR.Start)
+ n, err := h.writeFromBlocksAt(rw.ctx, gapSrcs, gapMR.Start)
done += n
rw.off += n
srcs = srcs.DropFirst64(n)
@@ -527,7 +536,7 @@ exitLoop:
if err := fsutil.SyncDirty(rw.ctx, memmap.MappableRange{
Start: start,
End: rw.off,
- }, &rw.d.cache, &rw.d.dirty, rw.d.size, mf, rw.d.handle.writeFromBlocksAt); err != nil {
+ }, &rw.d.cache, &rw.d.dirty, rw.d.size, mf, h.writeFromBlocksAt); err != nil {
// We have no idea how many bytes were actually flushed.
rw.off = start
done = 0
@@ -545,6 +554,7 @@ func (d *dentry) writeback(ctx context.Context, offset, size int64) error {
}
d.handleMu.RLock()
defer d.handleMu.RUnlock()
+ h := d.writeHandleLocked()
d.dataMu.Lock()
defer d.dataMu.Unlock()
// Compute the range of valid bytes (overflow-checked).
@@ -558,7 +568,7 @@ func (d *dentry) writeback(ctx context.Context, offset, size int64) error {
return fsutil.SyncDirty(ctx, memmap.MappableRange{
Start: uint64(offset),
End: uint64(end),
- }, &d.cache, &d.dirty, d.size, d.fs.mfp.MemoryFile(), d.handle.writeFromBlocksAt)
+ }, &d.cache, &d.dirty, d.size, d.fs.mfp.MemoryFile(), h.writeFromBlocksAt)
}
// Seek implements vfs.FileDescriptionImpl.Seek.
@@ -615,24 +625,23 @@ 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().syncSharedHandle(ctx)
+ return fd.dentry().syncCachedFile(ctx)
}
-func (d *dentry) syncSharedHandle(ctx context.Context) error {
+func (d *dentry) syncCachedFile(ctx context.Context) error {
d.handleMu.RLock()
defer d.handleMu.RUnlock()
- if d.handleWritable {
+ 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(), d.handle.writeFromBlocksAt)
+ err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, d.fs.mfp.MemoryFile(), h.writeFromBlocksAt)
d.dataMu.Unlock()
if err != nil {
return err
}
}
- // Sync the remote file.
- return d.handle.sync(ctx)
+ return d.syncRemoteFileLocked(ctx)
}
// ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap.
@@ -656,7 +665,7 @@ func (fd *regularFileFD) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpt
return syserror.ENODEV
}
d.handleMu.RLock()
- haveFD := d.handle.fd >= 0
+ haveFD := d.hostFD >= 0
d.handleMu.RUnlock()
if !haveFD {
return syserror.ENODEV
@@ -677,7 +686,7 @@ func (d *dentry) mayCachePages() bool {
return true
}
d.handleMu.RLock()
- haveFD := d.handle.fd >= 0
+ haveFD := d.hostFD >= 0
d.handleMu.RUnlock()
return haveFD
}
@@ -735,7 +744,7 @@ func (d *dentry) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR,
// Translate implements memmap.Mappable.Translate.
func (d *dentry) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
d.handleMu.RLock()
- if d.handle.fd >= 0 && !d.fs.opts.forcePageCache {
+ if d.hostFD >= 0 && !d.fs.opts.forcePageCache {
d.handleMu.RUnlock()
mr := optional
if d.fs.opts.limitHostFDTranslation {
@@ -771,7 +780,8 @@ func (d *dentry) Translate(ctx context.Context, required, optional memmap.Mappab
}
mf := d.fs.mfp.MemoryFile()
- cerr := d.cache.Fill(ctx, required, maxFillRange(required, optional), mf, usage.PageCache, d.handle.readToBlocksAt)
+ h := d.readHandleLocked()
+ cerr := d.cache.Fill(ctx, required, maxFillRange(required, optional), mf, usage.PageCache, h.readToBlocksAt)
var ts []memmap.Translation
var translatedEnd uint64
@@ -840,9 +850,12 @@ func (d *dentry) InvalidateUnsavable(ctx context.Context) error {
// Write the cache's contents back to the remote file so that if we have a
// host fd after restore, the remote file's contents are coherent.
mf := d.fs.mfp.MemoryFile()
+ d.handleMu.RLock()
+ defer d.handleMu.RUnlock()
+ h := d.writeHandleLocked()
d.dataMu.Lock()
defer d.dataMu.Unlock()
- if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, mf, d.handle.writeFromBlocksAt); err != nil {
+ if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, mf, h.writeFromBlocksAt); err != nil {
return err
}
@@ -857,20 +870,23 @@ func (d *dentry) InvalidateUnsavable(ctx context.Context) error {
// Evict implements pgalloc.EvictableMemoryUser.Evict.
func (d *dentry) Evict(ctx context.Context, er pgalloc.EvictableRange) {
+ mr := memmap.MappableRange{er.Start, er.End}
+ mf := d.fs.mfp.MemoryFile()
d.mapsMu.Lock()
defer d.mapsMu.Unlock()
+ d.handleMu.RLock()
+ defer d.handleMu.RUnlock()
+ h := d.writeHandleLocked()
d.dataMu.Lock()
defer d.dataMu.Unlock()
- mr := memmap.MappableRange{er.Start, er.End}
- mf := d.fs.mfp.MemoryFile()
// Only allow pages that are no longer memory-mapped to be evicted.
for mgap := d.mappings.LowerBoundGap(mr.Start); mgap.Ok() && mgap.Start() < mr.End; mgap = mgap.NextGap() {
mgapMR := mgap.Range().Intersect(mr)
if mgapMR.Length() == 0 {
continue
}
- if err := fsutil.SyncDirty(ctx, mgapMR, &d.cache, &d.dirty, d.size, mf, d.handle.writeFromBlocksAt); err != nil {
+ if err := fsutil.SyncDirty(ctx, mgapMR, &d.cache, &d.dirty, d.size, mf, h.writeFromBlocksAt); err != nil {
log.Warningf("Failed to writeback cached data %v: %v", mgapMR, err)
}
d.cache.Drop(mgapMR, mf)
@@ -882,8 +898,8 @@ func (d *dentry) Evict(ctx context.Context, er pgalloc.EvictableRange) {
// cannot implement both vfs.DentryImpl.IncRef and memmap.File.IncRef.
//
// dentryPlatformFile is only used when a host FD representing the remote file
-// is available (i.e. dentry.handle.fd >= 0), and that FD is used for
-// application memory mappings (i.e. !filesystem.opts.forcePageCache).
+// is available (i.e. dentry.hostFD >= 0), and that FD is used for application
+// memory mappings (i.e. !filesystem.opts.forcePageCache).
type dentryPlatformFile struct {
*dentry
@@ -891,8 +907,8 @@ type dentryPlatformFile struct {
// by dentry.dataMu.
fdRefs fsutil.FrameRefSet
- // If this dentry represents a regular file, and handle.fd >= 0,
- // hostFileMapper caches mappings of handle.fd.
+ // If this dentry represents a regular file, and dentry.hostFD >= 0,
+ // hostFileMapper caches mappings of dentry.hostFD.
hostFileMapper fsutil.HostFileMapper
// hostFileMapperInitOnce is used to lazily initialize hostFileMapper.
@@ -916,15 +932,13 @@ func (d *dentryPlatformFile) DecRef(fr memmap.FileRange) {
// MapInternal implements memmap.File.MapInternal.
func (d *dentryPlatformFile) MapInternal(fr memmap.FileRange, at usermem.AccessType) (safemem.BlockSeq, error) {
d.handleMu.RLock()
- bs, err := d.hostFileMapper.MapInternal(fr, int(d.handle.fd), at.Write)
- d.handleMu.RUnlock()
- return bs, err
+ defer d.handleMu.RUnlock()
+ return d.hostFileMapper.MapInternal(fr, int(d.hostFD), at.Write)
}
// FD implements memmap.File.FD.
func (d *dentryPlatformFile) FD() int {
d.handleMu.RLock()
- fd := d.handle.fd
- d.handleMu.RUnlock()
- return int(fd)
+ defer d.handleMu.RUnlock()
+ return int(d.hostFD)
}
diff --git a/pkg/sentry/fsimpl/gofer/special_file.go b/pkg/sentry/fsimpl/gofer/special_file.go
index fc269ef2b..a6368fdd0 100644
--- a/pkg/sentry/fsimpl/gofer/special_file.go
+++ b/pkg/sentry/fsimpl/gofer/special_file.go
@@ -17,6 +17,7 @@ package gofer
import (
"sync"
"sync/atomic"
+ "syscall"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
@@ -279,5 +280,13 @@ func (fd *specialFileFD) Seek(ctx context.Context, offset int64, whence int32) (
// Sync implements vfs.FileDescriptionImpl.Sync.
func (fd *specialFileFD) Sync(ctx context.Context) error {
- return fd.dentry().syncSharedHandle(ctx)
+ // 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)
}
diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go
index 576ab3920..d3c1197e3 100644
--- a/pkg/sentry/vfs/file_description.go
+++ b/pkg/sentry/vfs/file_description.go
@@ -356,6 +356,8 @@ type FileDescriptionImpl interface {
// Allocate grows the file to offset + length bytes.
// Only mode == 0 is supported currently.
+ //
+ // Preconditions: The FileDescription was opened for writing.
Allocate(ctx context.Context, mode, offset, length uint64) error
// waiter.Waitable methods may be used to poll for I/O events.
@@ -565,6 +567,9 @@ func (fd *FileDescription) StatFS(ctx context.Context) (linux.Statfs, error) {
// Allocate grows file represented by FileDescription to offset + length bytes.
func (fd *FileDescription) Allocate(ctx context.Context, mode, offset, length uint64) error {
+ if !fd.IsWritable() {
+ return syserror.EBADF
+ }
return fd.impl.Allocate(ctx, mode, offset, length)
}