summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl/host
diff options
context:
space:
mode:
authorJamie Liu <jamieliu@google.com>2020-10-23 17:46:43 -0700
committergVisor bot <gvisor-bot@google.com>2020-10-23 17:48:33 -0700
commit9f87400f087df0492cf181c97f431b6d5ce3a987 (patch)
treee52617b64a20a84db67f1bfe34907677571b92c7 /pkg/sentry/fsimpl/host
parent634e14a09408e50ef70442c0114a8b1dd12c8d03 (diff)
Support VFS2 save/restore.
Inode number consistency checks are now skipped in save/restore tests for reasons described in greatest detail in StatTest.StateDoesntChangeAfterRename. They pass in VFS1 due to the bug described in new test case SimpleStatTest.DifferentFilesHaveDifferentDeviceInodeNumberPairs. Fixes #1663 PiperOrigin-RevId: 338776148
Diffstat (limited to 'pkg/sentry/fsimpl/host')
-rw-r--r--pkg/sentry/fsimpl/host/BUILD1
-rw-r--r--pkg/sentry/fsimpl/host/control.go2
-rw-r--r--pkg/sentry/fsimpl/host/host.go249
-rw-r--r--pkg/sentry/fsimpl/host/mmap.go2
-rw-r--r--pkg/sentry/fsimpl/host/save_restore.go78
-rw-r--r--pkg/sentry/fsimpl/host/util.go6
6 files changed, 233 insertions, 105 deletions
diff --git a/pkg/sentry/fsimpl/host/BUILD b/pkg/sentry/fsimpl/host/BUILD
index 8a3e5c81e..dc0f86061 100644
--- a/pkg/sentry/fsimpl/host/BUILD
+++ b/pkg/sentry/fsimpl/host/BUILD
@@ -34,6 +34,7 @@ go_library(
"inode_refs.go",
"ioctl_unsafe.go",
"mmap.go",
+ "save_restore.go",
"socket.go",
"socket_iovec.go",
"socket_unsafe.go",
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/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 {