summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl/host
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fsimpl/host')
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/control.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/host.go132
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/host_state_autogen.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/host_unsafe_state_autogen.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/ioctl_unsafe.go0
-rw-r--r--pkg/sentry/fsimpl/host/mmap.go132
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/socket.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/socket_iovec.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/socket_unsafe.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/tty.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/util.go0
-rw-r--r--[-rwxr-xr-x]pkg/sentry/fsimpl/host/util_unsafe.go0
12 files changed, 211 insertions, 53 deletions
diff --git a/pkg/sentry/fsimpl/host/control.go b/pkg/sentry/fsimpl/host/control.go
index b9082a20f..b9082a20f 100755..100644
--- a/pkg/sentry/fsimpl/host/control.go
+++ b/pkg/sentry/fsimpl/host/control.go
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index 8caf55a1b..18b127521 100755..100644
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -86,15 +86,13 @@ func NewFD(ctx context.Context, mnt *vfs.Mount, hostFD int, opts *NewFDOptions)
i := &inode{
hostFD: hostFD,
- seekable: seekable,
+ ino: fs.NextIno(),
isTTY: opts.IsTTY,
- canMap: canMap(uint32(fileType)),
wouldBlock: wouldBlock(uint32(fileType)),
- ino: fs.NextIno(),
- // For simplicity, set offset to 0. Technically, we should use the existing
- // offset on the host if the file is seekable.
- offset: 0,
+ seekable: seekable,
+ canMap: canMap(uint32(fileType)),
}
+ i.pf.inode = i
// Non-seekable files can't be memory mapped, assert this.
if !i.seekable && i.canMap {
@@ -117,6 +115,10 @@ func NewFD(ctx context.Context, mnt *vfs.Mount, hostFD int, opts *NewFDOptions)
// i.open will take a reference on d.
defer d.DecRef()
+
+ // For simplicity, fileDescription.offset is set to 0. Technically, we
+ // should only set to 0 on files that are not seekable (sockets, pipes,
+ // etc.), and use the offset from the host fd otherwise when importing.
return i.open(ctx, d.VFSDentry(), mnt, flags)
}
@@ -189,11 +191,15 @@ type inode struct {
// This field is initialized at creation time and is immutable.
hostFD int
- // wouldBlock is true if the host FD would return EWOULDBLOCK for
- // operations that would block.
+ // ino is an inode number unique within this filesystem.
//
// This field is initialized at creation time and is immutable.
- wouldBlock bool
+ 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
@@ -202,29 +208,29 @@ type inode struct {
// This field is initialized at creation time and is immutable.
seekable bool
- // isTTY is true if this file represents a TTY.
+ // 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.
- isTTY bool
+ 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
- // ino is an inode number unique within this filesystem.
- //
- // This field is initialized at creation time and is immutable.
- ino uint64
+ // mapsMu protects mappings.
+ mapsMu sync.Mutex
- // offsetMu protects offset.
- offsetMu sync.Mutex
-
- // offset specifies the current file offset.
- offset int64
+ // If canMap is true, mappings tracks mappings of hostFD into
+ // memmap.MappingSpaces.
+ mappings memmap.MappingSet
- // Event queue for blocking operations.
- queue waiter.Queue
+ // pf implements platform.File for mappings of hostFD.
+ pf inodePlatformFile
}
// CheckPermissions implements kernfs.Inode.
@@ -388,6 +394,21 @@ func (i *inode) SetStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Cre
if err := syscall.Ftruncate(i.hostFD, int64(s.Size)); err != nil {
return err
}
+ oldSize := uint64(hostStat.Size)
+ if s.Size < oldSize {
+ oldpgend, _ := usermem.PageRoundUp(oldSize)
+ newpgend, _ := usermem.PageRoundUp(s.Size)
+ if oldpgend != newpgend {
+ i.mapsMu.Lock()
+ i.mappings.Invalidate(memmap.MappableRange{newpgend, oldpgend}, memmap.InvalidateOpts{
+ // Compare Linux's mm/truncate.c:truncate_setsize() =>
+ // truncate_pagecache() =>
+ // mm/memory.c:unmap_mapping_range(evencows=1).
+ InvalidatePrivate: true,
+ })
+ i.mapsMu.Unlock()
+ }
+ }
}
if m&(linux.STATX_ATIME|linux.STATX_MTIME) != 0 {
ts := [2]syscall.Timespec{
@@ -464,9 +485,6 @@ func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount, flags u
return vfsfd, nil
}
- // For simplicity, set offset to 0. Technically, we should
- // only set to 0 on files that are not seekable (sockets, pipes, etc.),
- // and use the offset from the host fd otherwise.
fd := &fileDescription{inode: i}
vfsfd := &fd.vfsfd
if err := vfsfd.Init(fd, flags, mnt, d, &vfs.FileDescriptionOptions{}); err != nil {
@@ -487,6 +505,13 @@ type fileDescription struct {
//
// inode is immutable after fileDescription creation.
inode *inode
+
+ // offsetMu protects offset.
+ offsetMu sync.Mutex
+
+ // offset specifies the current file offset. It is only meaningful when
+ // inode.seekable is true.
+ offset int64
}
// SetStat implements vfs.FileDescriptionImpl.
@@ -532,10 +557,10 @@ func (f *fileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts
return n, err
}
// TODO(gvisor.dev/issue/1672): Cache pages, when forced to do so.
- i.offsetMu.Lock()
- n, err := readFromHostFD(ctx, i.hostFD, dst, i.offset, opts.Flags)
- i.offset += n
- i.offsetMu.Unlock()
+ f.offsetMu.Lock()
+ n, err := readFromHostFD(ctx, i.hostFD, dst, f.offset, opts.Flags)
+ f.offset += n
+ f.offsetMu.Unlock()
return n, err
}
@@ -572,10 +597,10 @@ func (f *fileDescription) Write(ctx context.Context, src usermem.IOSequence, opt
}
// TODO(gvisor.dev/issue/1672): Cache pages, when forced to do so.
// TODO(gvisor.dev/issue/1672): Write to end of file and update offset if O_APPEND is set on this file.
- i.offsetMu.Lock()
- n, err := writeToHostFD(ctx, i.hostFD, src, i.offset, opts.Flags)
- i.offset += n
- i.offsetMu.Unlock()
+ f.offsetMu.Lock()
+ n, err := writeToHostFD(ctx, i.hostFD, src, f.offset, opts.Flags)
+ f.offset += n
+ f.offsetMu.Unlock()
return n, err
}
@@ -600,41 +625,41 @@ func (f *fileDescription) Seek(_ context.Context, offset int64, whence int32) (i
return 0, syserror.ESPIPE
}
- i.offsetMu.Lock()
- defer i.offsetMu.Unlock()
+ f.offsetMu.Lock()
+ defer f.offsetMu.Unlock()
switch whence {
case linux.SEEK_SET:
if offset < 0 {
- return i.offset, syserror.EINVAL
+ return f.offset, syserror.EINVAL
}
- i.offset = offset
+ f.offset = offset
case linux.SEEK_CUR:
- // Check for overflow. Note that underflow cannot occur, since i.offset >= 0.
- if offset > math.MaxInt64-i.offset {
- return i.offset, syserror.EOVERFLOW
+ // Check for overflow. Note that underflow cannot occur, since f.offset >= 0.
+ if offset > math.MaxInt64-f.offset {
+ return f.offset, syserror.EOVERFLOW
}
- if i.offset+offset < 0 {
- return i.offset, syserror.EINVAL
+ if f.offset+offset < 0 {
+ return f.offset, syserror.EINVAL
}
- i.offset += offset
+ f.offset += offset
case linux.SEEK_END:
var s syscall.Stat_t
if err := syscall.Fstat(i.hostFD, &s); err != nil {
- return i.offset, err
+ return f.offset, err
}
size := s.Size
// Check for overflow. Note that underflow cannot occur, since size >= 0.
if offset > math.MaxInt64-size {
- return i.offset, syserror.EOVERFLOW
+ return f.offset, syserror.EOVERFLOW
}
if size+offset < 0 {
- return i.offset, syserror.EINVAL
+ return f.offset, syserror.EINVAL
}
- i.offset = size + offset
+ f.offset = size + offset
case linux.SEEK_DATA, linux.SEEK_HOLE:
// Modifying the offset in the host file table should not matter, since
@@ -643,16 +668,16 @@ func (f *fileDescription) Seek(_ context.Context, offset int64, whence int32) (i
// For reading and writing, we always rely on our internal offset.
n, err := unix.Seek(i.hostFD, offset, int(whence))
if err != nil {
- return i.offset, err
+ return f.offset, err
}
- i.offset = n
+ f.offset = n
default:
// Invalid whence.
- return i.offset, syserror.EINVAL
+ return f.offset, syserror.EINVAL
}
- return i.offset, nil
+ return f.offset, nil
}
// Sync implements FileDescriptionImpl.
@@ -666,8 +691,9 @@ func (f *fileDescription) ConfigureMMap(_ context.Context, opts *memmap.MMapOpts
if !f.inode.canMap {
return syserror.ENODEV
}
- // TODO(gvisor.dev/issue/1672): Implement ConfigureMMap and Mappable interface.
- return syserror.ENODEV
+ i := f.inode
+ i.pf.fileMapperInitOnce.Do(i.pf.fileMapper.Init)
+ return vfs.GenericConfigureMMap(&f.vfsfd, i, opts)
}
// EventRegister implements waiter.Waitable.EventRegister.
diff --git a/pkg/sentry/fsimpl/host/host_state_autogen.go b/pkg/sentry/fsimpl/host/host_state_autogen.go
index 28bc79d04..28bc79d04 100755..100644
--- a/pkg/sentry/fsimpl/host/host_state_autogen.go
+++ b/pkg/sentry/fsimpl/host/host_state_autogen.go
diff --git a/pkg/sentry/fsimpl/host/host_unsafe_state_autogen.go b/pkg/sentry/fsimpl/host/host_unsafe_state_autogen.go
index b2d8c661f..b2d8c661f 100755..100644
--- a/pkg/sentry/fsimpl/host/host_unsafe_state_autogen.go
+++ b/pkg/sentry/fsimpl/host/host_unsafe_state_autogen.go
diff --git a/pkg/sentry/fsimpl/host/ioctl_unsafe.go b/pkg/sentry/fsimpl/host/ioctl_unsafe.go
index 0983bf7d8..0983bf7d8 100755..100644
--- a/pkg/sentry/fsimpl/host/ioctl_unsafe.go
+++ b/pkg/sentry/fsimpl/host/ioctl_unsafe.go
diff --git a/pkg/sentry/fsimpl/host/mmap.go b/pkg/sentry/fsimpl/host/mmap.go
new file mode 100644
index 000000000..8545a82f0
--- /dev/null
+++ b/pkg/sentry/fsimpl/host/mmap.go
@@ -0,0 +1,132 @@
+// 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 (
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/safemem"
+ "gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
+ "gvisor.dev/gvisor/pkg/sentry/memmap"
+ "gvisor.dev/gvisor/pkg/sentry/platform"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+// inodePlatformFile implements platform.File. It exists solely because inode
+// cannot implement both kernfs.Inode.IncRef and platform.File.IncRef.
+//
+// inodePlatformFile should only be used if inode.canMap is true.
+type inodePlatformFile struct {
+ *inode
+
+ // fdRefsMu protects fdRefs.
+ fdRefsMu sync.Mutex
+
+ // fdRefs counts references on platform.File offsets. It is used solely for
+ // memory accounting.
+ fdRefs fsutil.FrameRefSet
+
+ // fileMapper caches mappings of the host file represented by this inode.
+ fileMapper fsutil.HostFileMapper
+
+ // fileMapperInitOnce is used to lazily initialize fileMapper.
+ fileMapperInitOnce sync.Once
+}
+
+// IncRef implements platform.File.IncRef.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inodePlatformFile) IncRef(fr platform.FileRange) {
+ i.fdRefsMu.Lock()
+ i.fdRefs.IncRefAndAccount(fr)
+ i.fdRefsMu.Unlock()
+}
+
+// DecRef implements platform.File.DecRef.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inodePlatformFile) DecRef(fr platform.FileRange) {
+ i.fdRefsMu.Lock()
+ i.fdRefs.DecRefAndAccount(fr)
+ i.fdRefsMu.Unlock()
+}
+
+// MapInternal implements platform.File.MapInternal.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inodePlatformFile) MapInternal(fr platform.FileRange, at usermem.AccessType) (safemem.BlockSeq, error) {
+ return i.fileMapper.MapInternal(fr, i.hostFD, at.Write)
+}
+
+// FD implements platform.File.FD.
+func (i *inodePlatformFile) FD() int {
+ return i.hostFD
+}
+
+// AddMapping implements memmap.Mappable.AddMapping.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inode) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) error {
+ i.mapsMu.Lock()
+ mapped := i.mappings.AddMapping(ms, ar, offset, writable)
+ for _, r := range mapped {
+ i.pf.fileMapper.IncRefOn(r)
+ }
+ i.mapsMu.Unlock()
+ return nil
+}
+
+// RemoveMapping implements memmap.Mappable.RemoveMapping.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inode) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) {
+ i.mapsMu.Lock()
+ unmapped := i.mappings.RemoveMapping(ms, ar, offset, writable)
+ for _, r := range unmapped {
+ i.pf.fileMapper.DecRefOn(r)
+ }
+ i.mapsMu.Unlock()
+}
+
+// CopyMapping implements memmap.Mappable.CopyMapping.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inode) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64, writable bool) error {
+ return i.AddMapping(ctx, ms, dstAR, offset, writable)
+}
+
+// Translate implements memmap.Mappable.Translate.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inode) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
+ mr := optional
+ return []memmap.Translation{
+ {
+ Source: mr,
+ File: &i.pf,
+ Offset: mr.Start,
+ Perms: usermem.AnyAccess,
+ },
+ }, nil
+}
+
+// InvalidateUnsavable implements memmap.Mappable.InvalidateUnsavable.
+//
+// Precondition: i.inode.canMap must be true.
+func (i *inode) InvalidateUnsavable(ctx context.Context) error {
+ // We expect the same host fd across save/restore, so all translations
+ // should be valid.
+ return nil
+}
diff --git a/pkg/sentry/fsimpl/host/socket.go b/pkg/sentry/fsimpl/host/socket.go
index 38f1fbfba..38f1fbfba 100755..100644
--- a/pkg/sentry/fsimpl/host/socket.go
+++ b/pkg/sentry/fsimpl/host/socket.go
diff --git a/pkg/sentry/fsimpl/host/socket_iovec.go b/pkg/sentry/fsimpl/host/socket_iovec.go
index 584c247d2..584c247d2 100755..100644
--- a/pkg/sentry/fsimpl/host/socket_iovec.go
+++ b/pkg/sentry/fsimpl/host/socket_iovec.go
diff --git a/pkg/sentry/fsimpl/host/socket_unsafe.go b/pkg/sentry/fsimpl/host/socket_unsafe.go
index 35ded24bc..35ded24bc 100755..100644
--- a/pkg/sentry/fsimpl/host/socket_unsafe.go
+++ b/pkg/sentry/fsimpl/host/socket_unsafe.go
diff --git a/pkg/sentry/fsimpl/host/tty.go b/pkg/sentry/fsimpl/host/tty.go
index 68af6e5af..68af6e5af 100755..100644
--- a/pkg/sentry/fsimpl/host/tty.go
+++ b/pkg/sentry/fsimpl/host/tty.go
diff --git a/pkg/sentry/fsimpl/host/util.go b/pkg/sentry/fsimpl/host/util.go
index 2bc757b1a..2bc757b1a 100755..100644
--- a/pkg/sentry/fsimpl/host/util.go
+++ b/pkg/sentry/fsimpl/host/util.go
diff --git a/pkg/sentry/fsimpl/host/util_unsafe.go b/pkg/sentry/fsimpl/host/util_unsafe.go
index 5136ac844..5136ac844 100755..100644
--- a/pkg/sentry/fsimpl/host/util_unsafe.go
+++ b/pkg/sentry/fsimpl/host/util_unsafe.go