summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fsimpl')
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/BUILD48
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/base.go261
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cgroupfs.go425
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpu.go70
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpuacct.go114
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpuset.go39
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/job.go64
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/memory.go74
-rw-r--r--pkg/sentry/fsimpl/eventfd/BUILD1
-rw-r--r--pkg/sentry/fsimpl/eventfd/eventfd.go7
-rw-r--r--pkg/sentry/fsimpl/fuse/BUILD2
-rw-r--r--pkg/sentry/fsimpl/fuse/read_write.go20
-rw-r--r--pkg/sentry/fsimpl/fuse/request_response.go20
-rw-r--r--pkg/sentry/fsimpl/fuse/utils_test.go13
-rw-r--r--pkg/sentry/fsimpl/gofer/BUILD1
-rw-r--r--pkg/sentry/fsimpl/gofer/directory.go4
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go39
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go252
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer_test.go6
-rw-r--r--pkg/sentry/fsimpl/gofer/regular_file.go39
-rw-r--r--pkg/sentry/fsimpl/gofer/save_restore.go4
-rw-r--r--pkg/sentry/fsimpl/host/BUILD1
-rw-r--r--pkg/sentry/fsimpl/host/host.go8
-rw-r--r--pkg/sentry/fsimpl/host/save_restore.go11
-rw-r--r--pkg/sentry/fsimpl/host/socket.go19
-rw-r--r--pkg/sentry/fsimpl/kernfs/BUILD1
-rw-r--r--pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go10
-rw-r--r--pkg/sentry/fsimpl/kernfs/inode_impl_util.go4
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs.go10
-rw-r--r--pkg/sentry/fsimpl/kernfs/mmap_util.go14
-rw-r--r--pkg/sentry/fsimpl/overlay/BUILD1
-rw-r--r--pkg/sentry/fsimpl/overlay/copy_up.go6
-rw-r--r--pkg/sentry/fsimpl/overlay/regular_file.go9
-rw-r--r--pkg/sentry/fsimpl/pipefs/BUILD2
-rw-r--r--pkg/sentry/fsimpl/pipefs/pipefs.go4
-rw-r--r--pkg/sentry/fsimpl/proc/BUILD1
-rw-r--r--pkg/sentry/fsimpl/proc/filesystem.go6
-rw-r--r--pkg/sentry/fsimpl/proc/task.go23
-rw-r--r--pkg/sentry/fsimpl/proc/task_files.go60
-rw-r--r--pkg/sentry/fsimpl/proc/task_net.go20
-rw-r--r--pkg/sentry/fsimpl/proc/tasks.go19
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_files.go20
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_sys.go11
-rw-r--r--pkg/sentry/fsimpl/proc/yama.go3
-rw-r--r--pkg/sentry/fsimpl/sys/sys.go8
-rw-r--r--pkg/sentry/fsimpl/testutil/BUILD1
-rw-r--r--pkg/sentry/fsimpl/testutil/testutil.go4
-rw-r--r--pkg/sentry/fsimpl/timerfd/BUILD1
-rw-r--r--pkg/sentry/fsimpl/timerfd/timerfd.go3
-rw-r--r--pkg/sentry/fsimpl/tmpfs/BUILD1
-rw-r--r--pkg/sentry/fsimpl/tmpfs/regular_file.go26
-rw-r--r--pkg/sentry/fsimpl/tmpfs/tmpfs.go14
-rw-r--r--pkg/sentry/fsimpl/verity/BUILD3
-rw-r--r--pkg/sentry/fsimpl/verity/filesystem.go102
-rw-r--r--pkg/sentry/fsimpl/verity/verity.go314
-rw-r--r--pkg/sentry/fsimpl/verity/verity_test.go3
56 files changed, 1885 insertions, 361 deletions
diff --git a/pkg/sentry/fsimpl/cgroupfs/BUILD b/pkg/sentry/fsimpl/cgroupfs/BUILD
new file mode 100644
index 000000000..37efb641a
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/BUILD
@@ -0,0 +1,48 @@
+load("//tools:defs.bzl", "go_library")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
+
+licenses(["notice"])
+
+go_template_instance(
+ name = "dir_refs",
+ out = "dir_refs.go",
+ package = "cgroupfs",
+ prefix = "dir",
+ template = "//pkg/refsvfs2:refs_template",
+ types = {
+ "T": "dir",
+ },
+)
+
+go_library(
+ name = "cgroupfs",
+ srcs = [
+ "base.go",
+ "cgroupfs.go",
+ "cpu.go",
+ "cpuacct.go",
+ "cpuset.go",
+ "dir_refs.go",
+ "job.go",
+ "memory.go",
+ ],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/coverage",
+ "//pkg/log",
+ "//pkg/refs",
+ "//pkg/refsvfs2",
+ "//pkg/sentry/arch",
+ "//pkg/sentry/fsimpl/kernfs",
+ "//pkg/sentry/kernel",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/memmap",
+ "//pkg/sentry/usage",
+ "//pkg/sentry/vfs",
+ "//pkg/sync",
+ "//pkg/syserror",
+ "//pkg/usermem",
+ ],
+)
diff --git a/pkg/sentry/fsimpl/cgroupfs/base.go b/pkg/sentry/fsimpl/cgroupfs/base.go
new file mode 100644
index 000000000..0f54888d8
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/base.go
@@ -0,0 +1,261 @@
+// Copyright 2021 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 cgroupfs
+
+import (
+ "bytes"
+ "fmt"
+ "sort"
+ "strconv"
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+// controllerCommon implements kernel.CgroupController.
+//
+// Must call init before use.
+//
+// +stateify savable
+type controllerCommon struct {
+ ty kernel.CgroupControllerType
+ fs *filesystem
+}
+
+func (c *controllerCommon) init(ty kernel.CgroupControllerType, fs *filesystem) {
+ c.ty = ty
+ c.fs = fs
+}
+
+// Type implements kernel.CgroupController.Type.
+func (c *controllerCommon) Type() kernel.CgroupControllerType {
+ return kernel.CgroupControllerType(c.ty)
+}
+
+// HierarchyID implements kernel.CgroupController.HierarchyID.
+func (c *controllerCommon) HierarchyID() uint32 {
+ return c.fs.hierarchyID
+}
+
+// NumCgroups implements kernel.CgroupController.NumCgroups.
+func (c *controllerCommon) NumCgroups() uint64 {
+ return atomic.LoadUint64(&c.fs.numCgroups)
+}
+
+// Enabled implements kernel.CgroupController.Enabled.
+//
+// Controllers are currently always enabled.
+func (c *controllerCommon) Enabled() bool {
+ return true
+}
+
+// Filesystem implements kernel.CgroupController.Filesystem.
+func (c *controllerCommon) Filesystem() *vfs.Filesystem {
+ return c.fs.VFSFilesystem()
+}
+
+// RootCgroup implements kernel.CgroupController.RootCgroup.
+func (c *controllerCommon) RootCgroup() kernel.Cgroup {
+ return c.fs.rootCgroup()
+}
+
+// controller is an interface for common functionality related to all cgroups.
+// It is an extension of the public cgroup interface, containing cgroup
+// functionality private to cgroupfs.
+type controller interface {
+ kernel.CgroupController
+
+ // AddControlFiles should extend the contents map with inodes representing
+ // control files defined by this controller.
+ AddControlFiles(ctx context.Context, creds *auth.Credentials, c *cgroupInode, contents map[string]kernfs.Inode)
+}
+
+// cgroupInode implements kernel.CgroupImpl and kernfs.Inode.
+//
+// +stateify savable
+type cgroupInode struct {
+ dir
+ fs *filesystem
+
+ // ts is the list of tasks in this cgroup. The kernel is responsible for
+ // removing tasks from this list before they're destroyed, so any tasks on
+ // this list are always valid.
+ //
+ // ts, and cgroup membership in general is protected by fs.tasksMu.
+ ts map[*kernel.Task]struct{}
+}
+
+var _ kernel.CgroupImpl = (*cgroupInode)(nil)
+
+func (fs *filesystem) newCgroupInode(ctx context.Context, creds *auth.Credentials) kernfs.Inode {
+ c := &cgroupInode{
+ fs: fs,
+ ts: make(map[*kernel.Task]struct{}),
+ }
+
+ contents := make(map[string]kernfs.Inode)
+ contents["cgroup.procs"] = fs.newControllerFile(ctx, creds, &cgroupProcsData{c})
+ contents["tasks"] = fs.newControllerFile(ctx, creds, &tasksData{c})
+
+ for _, ctl := range fs.controllers {
+ ctl.AddControlFiles(ctx, creds, c, contents)
+ }
+
+ c.dir.InodeAttrs.Init(ctx, creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|linux.FileMode(0555))
+ c.dir.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
+ c.dir.InitRefs()
+ c.dir.IncLinks(c.dir.OrderedChildren.Populate(contents))
+
+ atomic.AddUint64(&fs.numCgroups, 1)
+
+ return c
+}
+
+func (c *cgroupInode) HierarchyID() uint32 {
+ return c.fs.hierarchyID
+}
+
+// Controllers implements kernel.CgroupImpl.Controllers.
+func (c *cgroupInode) Controllers() []kernel.CgroupController {
+ return c.fs.kcontrollers
+}
+
+// Enter implements kernel.CgroupImpl.Enter.
+func (c *cgroupInode) Enter(t *kernel.Task) {
+ c.fs.tasksMu.Lock()
+ c.ts[t] = struct{}{}
+ c.fs.tasksMu.Unlock()
+}
+
+// Leave implements kernel.CgroupImpl.Leave.
+func (c *cgroupInode) Leave(t *kernel.Task) {
+ c.fs.tasksMu.Lock()
+ delete(c.ts, t)
+ c.fs.tasksMu.Unlock()
+}
+
+func sortTIDs(tids []kernel.ThreadID) {
+ sort.Slice(tids, func(i, j int) bool { return tids[i] < tids[j] })
+}
+
+// +stateify savable
+type cgroupProcsData struct {
+ *cgroupInode
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *cgroupProcsData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ t := kernel.TaskFromContext(ctx)
+ currPidns := t.ThreadGroup().PIDNamespace()
+
+ pgids := make(map[kernel.ThreadID]struct{})
+
+ d.fs.tasksMu.RLock()
+ defer d.fs.tasksMu.RUnlock()
+
+ for task := range d.ts {
+ // Map dedups pgid, since iterating over all tasks produces multiple
+ // entries for the group leaders.
+ if pgid := currPidns.IDOfThreadGroup(task.ThreadGroup()); pgid != 0 {
+ pgids[pgid] = struct{}{}
+ }
+ }
+
+ pgidList := make([]kernel.ThreadID, 0, len(pgids))
+ for pgid, _ := range pgids {
+ pgidList = append(pgidList, pgid)
+ }
+ sortTIDs(pgidList)
+
+ for _, pgid := range pgidList {
+ fmt.Fprintf(buf, "%d\n", pgid)
+ }
+
+ return nil
+}
+
+// Write implements vfs.WritableDynamicBytesSource.Write.
+func (d *cgroupProcsData) Write(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
+ // TODO(b/183137098): Payload is the pid for a process to add to this cgroup.
+ return src.NumBytes(), nil
+}
+
+// +stateify savable
+type tasksData struct {
+ *cgroupInode
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *tasksData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ t := kernel.TaskFromContext(ctx)
+ currPidns := t.ThreadGroup().PIDNamespace()
+
+ var pids []kernel.ThreadID
+
+ d.fs.tasksMu.RLock()
+ defer d.fs.tasksMu.RUnlock()
+
+ for task := range d.ts {
+ if pid := currPidns.IDOfTask(task); pid != 0 {
+ pids = append(pids, pid)
+ }
+ }
+ sortTIDs(pids)
+
+ for _, pid := range pids {
+ fmt.Fprintf(buf, "%d\n", pid)
+ }
+
+ return nil
+}
+
+// Write implements vfs.WritableDynamicBytesSource.Write.
+func (d *tasksData) Write(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
+ // TODO(b/183137098): Payload is the pid for a process to add to this cgroup.
+ return src.NumBytes(), nil
+}
+
+// parseInt64FromString interprets src as string encoding a int64 value, and
+// returns the parsed value.
+func parseInt64FromString(ctx context.Context, src usermem.IOSequence, offset int64) (val, len int64, err error) {
+ const maxInt64StrLen = 20 // i.e. len(fmt.Sprintf("%d", math.MinInt64)) == 20
+
+ t := kernel.TaskFromContext(ctx)
+ src = src.DropFirst64(offset)
+
+ buf := t.CopyScratchBuffer(maxInt64StrLen)
+ n, err := src.CopyIn(ctx, buf)
+ if err != nil {
+ return 0, int64(n), err
+ }
+ buf = buf[:n]
+
+ val, err = strconv.ParseInt(string(buf), 10, 64)
+ if err != nil {
+ // Note: This also handles zero-len writes if offset is beyond the end
+ // of src, or src is empty.
+ ctx.Warningf("cgroupfs.parseInt64FromString: failed to parse %q: %v", string(buf), err)
+ return 0, int64(n), syserror.EINVAL
+ }
+
+ return val, int64(n), nil
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go b/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go
new file mode 100644
index 000000000..bd3e69757
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go
@@ -0,0 +1,425 @@
+// Copyright 2021 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 cgroupfs implements cgroupfs.
+//
+// A cgroup is a collection of tasks on the system, organized into a tree-like
+// structure similar to a filesystem directory tree. In fact, each cgroup is
+// represented by a directory on cgroupfs, and is manipulated through control
+// files in the directory.
+//
+// All cgroups on a system are organized into hierarchies. Hierarchies are a
+// distinct tree of cgroups, with a common set of controllers. One or more
+// cgroupfs mounts may point to each hierarchy. These mounts provide a common
+// view into the same tree of cgroups.
+//
+// A controller (also known as a "resource controller", or a cgroup "subsystem")
+// determines the behaviour of each cgroup.
+//
+// In addition to cgroupfs, the kernel has a cgroup registry that tracks
+// system-wide state related to cgroups such as active hierarchies and the
+// controllers associated with them.
+//
+// Since cgroupfs doesn't allow hardlinks, there is a unique mapping between
+// cgroupfs dentries and inodes.
+//
+// # Synchronization
+//
+// Cgroup hierarchy creation and destruction is protected by the
+// kernel.CgroupRegistry.mu. Once created, a hierarchy's set of controllers, the
+// filesystem associated with it, and the root cgroup for the hierarchy are
+// immutable.
+//
+// Membership of tasks within cgroups is protected by
+// cgroupfs.filesystem.tasksMu. Tasks also maintain a set of all cgroups they're
+// in, and this list is protected by Task.mu.
+//
+// Lock order:
+//
+// kernel.CgroupRegistry.mu
+// cgroupfs.filesystem.mu
+// Task.mu
+// cgroupfs.filesystem.tasksMu.
+package cgroupfs
+
+import (
+ "fmt"
+ "sort"
+ "strconv"
+ "strings"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+const (
+ // Name is the default filesystem name.
+ Name = "cgroup"
+ readonlyFileMode = linux.FileMode(0444)
+ writableFileMode = linux.FileMode(0644)
+ defaultMaxCachedDentries = uint64(1000)
+)
+
+const (
+ controllerCPU = kernel.CgroupControllerType("cpu")
+ controllerCPUAcct = kernel.CgroupControllerType("cpuacct")
+ controllerCPUSet = kernel.CgroupControllerType("cpuset")
+ controllerJob = kernel.CgroupControllerType("job")
+ controllerMemory = kernel.CgroupControllerType("memory")
+)
+
+var allControllers = []kernel.CgroupControllerType{
+ controllerCPU,
+ controllerCPUAcct,
+ controllerCPUSet,
+ controllerJob,
+ controllerMemory,
+}
+
+// SupportedMountOptions is the set of supported mount options for cgroupfs.
+var SupportedMountOptions = []string{"all", "cpu", "cpuacct", "cpuset", "job", "memory"}
+
+// FilesystemType implements vfs.FilesystemType.
+//
+// +stateify savable
+type FilesystemType struct{}
+
+// InternalData contains internal data passed in to the cgroupfs mount via
+// vfs.GetFilesystemOptions.InternalData.
+//
+// +stateify savable
+type InternalData struct {
+ DefaultControlValues map[string]int64
+}
+
+// filesystem implements vfs.FilesystemImpl.
+//
+// +stateify savable
+type filesystem struct {
+ kernfs.Filesystem
+ devMinor uint32
+
+ // hierarchyID is the id the cgroup registry assigns to this hierarchy. Has
+ // the value kernel.InvalidCgroupHierarchyID until the FS is fully
+ // initialized.
+ //
+ // hierarchyID is immutable after initialization.
+ hierarchyID uint32
+
+ // controllers and kcontrollers are both the list of controllers attached to
+ // this cgroupfs. Both lists are the same set of controllers, but typecast
+ // to different interfaces for convenience. Both must stay in sync, and are
+ // immutable.
+ controllers []controller
+ kcontrollers []kernel.CgroupController
+
+ numCgroups uint64 // Protected by atomic ops.
+
+ root *kernfs.Dentry
+
+ // tasksMu serializes task membership changes across all cgroups within a
+ // filesystem.
+ tasksMu sync.RWMutex `state:"nosave"`
+}
+
+// Name implements vfs.FilesystemType.Name.
+func (FilesystemType) Name() string {
+ return Name
+}
+
+// Release implements vfs.FilesystemType.Release.
+func (FilesystemType) Release(ctx context.Context) {}
+
+// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
+func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
+ devMinor, err := vfsObj.GetAnonBlockDevMinor()
+ if err != nil {
+ return nil, nil, err
+ }
+
+ mopts := vfs.GenericParseMountOptions(opts.Data)
+ maxCachedDentries := defaultMaxCachedDentries
+ if str, ok := mopts["dentry_cache_limit"]; ok {
+ delete(mopts, "dentry_cache_limit")
+ maxCachedDentries, err = strconv.ParseUint(str, 10, 64)
+ if err != nil {
+ ctx.Warningf("sys.FilesystemType.GetFilesystem: invalid dentry cache limit: dentry_cache_limit=%s", str)
+ return nil, nil, syserror.EINVAL
+ }
+ }
+
+ var wantControllers []kernel.CgroupControllerType
+ if _, ok := mopts["cpu"]; ok {
+ delete(mopts, "cpu")
+ wantControllers = append(wantControllers, controllerCPU)
+ }
+ if _, ok := mopts["cpuacct"]; ok {
+ delete(mopts, "cpuacct")
+ wantControllers = append(wantControllers, controllerCPUAcct)
+ }
+ if _, ok := mopts["cpuset"]; ok {
+ delete(mopts, "cpuset")
+ wantControllers = append(wantControllers, controllerCPUSet)
+ }
+ if _, ok := mopts["job"]; ok {
+ delete(mopts, "job")
+ wantControllers = append(wantControllers, controllerJob)
+ }
+ if _, ok := mopts["memory"]; ok {
+ delete(mopts, "memory")
+ wantControllers = append(wantControllers, controllerMemory)
+ }
+ if _, ok := mopts["all"]; ok {
+ if len(wantControllers) > 0 {
+ ctx.Debugf("cgroupfs.FilesystemType.GetFilesystem: other controllers specified with all: %v", wantControllers)
+ return nil, nil, syserror.EINVAL
+ }
+
+ delete(mopts, "all")
+ wantControllers = allControllers
+ }
+
+ if len(wantControllers) == 0 {
+ // Specifying no controllers implies all controllers.
+ wantControllers = allControllers
+ }
+
+ if len(mopts) != 0 {
+ ctx.Debugf("cgroupfs.FilesystemType.GetFilesystem: unknown options: %v", mopts)
+ return nil, nil, syserror.EINVAL
+ }
+
+ k := kernel.KernelFromContext(ctx)
+ r := k.CgroupRegistry()
+
+ // "It is not possible to mount the same controller against multiple
+ // cgroup hierarchies. For example, it is not possible to mount both
+ // the cpu and cpuacct controllers against one hierarchy, and to mount
+ // the cpu controller alone against another hierarchy." - man cgroups(7)
+ //
+ // Is there a hierarchy available with all the controllers we want? If so,
+ // this mount is a view into the same hierarchy.
+ //
+ // Note: we're guaranteed to have at least one requested controller, since
+ // no explicit controller name implies all controllers.
+ if vfsfs := r.FindHierarchy(wantControllers); vfsfs != nil {
+ fs := vfsfs.Impl().(*filesystem)
+ ctx.Debugf("cgroupfs.FilesystemType.GetFilesystem: mounting new view to hierarchy %v", fs.hierarchyID)
+ fs.root.IncRef()
+ return vfsfs, fs.root.VFSDentry(), nil
+ }
+
+ // No existing hierarchy with the exactly controllers found. Make a new
+ // one. Note that it's possible this mount creation is unsatisfiable, if one
+ // or more of the requested controllers are already on existing
+ // hierarchies. We'll find out about such collisions when we try to register
+ // the new hierarchy later.
+ fs := &filesystem{
+ devMinor: devMinor,
+ }
+ fs.MaxCachedDentries = maxCachedDentries
+ fs.VFSFilesystem().Init(vfsObj, &fsType, fs)
+
+ var defaults map[string]int64
+ if opts.InternalData != nil {
+ ctx.Debugf("cgroupfs.FilesystemType.GetFilesystem: default control values: %v", defaults)
+ defaults = opts.InternalData.(*InternalData).DefaultControlValues
+ }
+
+ for _, ty := range wantControllers {
+ var c controller
+ switch ty {
+ case controllerCPU:
+ c = newCPUController(fs, defaults)
+ case controllerCPUAcct:
+ c = newCPUAcctController(fs)
+ case controllerCPUSet:
+ c = newCPUSetController(fs)
+ case controllerJob:
+ c = newJobController(fs)
+ case controllerMemory:
+ c = newMemoryController(fs, defaults)
+ default:
+ panic(fmt.Sprintf("Unreachable: unknown cgroup controller %q", ty))
+ }
+ fs.controllers = append(fs.controllers, c)
+ }
+
+ if len(defaults) != 0 {
+ // Internal data is always provided at sentry startup and unused values
+ // indicate a problem with the sandbox config. Fail fast.
+ panic(fmt.Sprintf("cgroupfs.FilesystemType.GetFilesystem: unknown internal mount data: %v", defaults))
+ }
+
+ // Controllers usually appear in alphabetical order when displayed. Sort it
+ // here now, so it never needs to be sorted elsewhere.
+ sort.Slice(fs.controllers, func(i, j int) bool { return fs.controllers[i].Type() < fs.controllers[j].Type() })
+ fs.kcontrollers = make([]kernel.CgroupController, 0, len(fs.controllers))
+ for _, c := range fs.controllers {
+ fs.kcontrollers = append(fs.kcontrollers, c)
+ }
+
+ root := fs.newCgroupInode(ctx, creds)
+ var rootD kernfs.Dentry
+ rootD.InitRoot(&fs.Filesystem, root)
+ fs.root = &rootD
+
+ // Register controllers. The registry may be modified concurrently, so if we
+ // get an error, we raced with someone else who registered the same
+ // controllers first.
+ hid, err := r.Register(fs.kcontrollers)
+ if err != nil {
+ ctx.Infof("cgroupfs.FilesystemType.GetFilesystem: failed to register new hierarchy with controllers %v: %v", wantControllers, err)
+ rootD.DecRef(ctx)
+ fs.VFSFilesystem().DecRef(ctx)
+ return nil, nil, syserror.EBUSY
+ }
+ fs.hierarchyID = hid
+
+ // Move all existing tasks to the root of the new hierarchy.
+ k.PopulateNewCgroupHierarchy(fs.rootCgroup())
+
+ return fs.VFSFilesystem(), rootD.VFSDentry(), nil
+}
+
+func (fs *filesystem) rootCgroup() kernel.Cgroup {
+ return kernel.Cgroup{
+ Dentry: fs.root,
+ CgroupImpl: fs.root.Inode().(kernel.CgroupImpl),
+ }
+}
+
+// Release implements vfs.FilesystemImpl.Release.
+func (fs *filesystem) Release(ctx context.Context) {
+ k := kernel.KernelFromContext(ctx)
+ r := k.CgroupRegistry()
+
+ if fs.hierarchyID != kernel.InvalidCgroupHierarchyID {
+ k.ReleaseCgroupHierarchy(fs.hierarchyID)
+ r.Unregister(fs.hierarchyID)
+ }
+
+ fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
+ fs.Filesystem.Release(ctx)
+}
+
+// MountOptions implements vfs.FilesystemImpl.MountOptions.
+func (fs *filesystem) MountOptions() string {
+ var cnames []string
+ for _, c := range fs.controllers {
+ cnames = append(cnames, string(c.Type()))
+ }
+ return strings.Join(cnames, ",")
+}
+
+// +stateify savable
+type implStatFS struct{}
+
+// StatFS implements kernfs.Inode.StatFS.
+func (*implStatFS) StatFS(context.Context, *vfs.Filesystem) (linux.Statfs, error) {
+ return vfs.GenericStatFS(linux.CGROUP_SUPER_MAGIC), nil
+}
+
+// dir implements kernfs.Inode for a generic cgroup resource controller
+// directory. Specific controllers extend this to add their own functionality.
+//
+// +stateify savable
+type dir struct {
+ dirRefs
+ kernfs.InodeAlwaysValid
+ kernfs.InodeAttrs
+ kernfs.InodeNotSymlink
+ kernfs.InodeDirectoryNoNewChildren // TODO(b/183137098): Implement mkdir.
+ kernfs.OrderedChildren
+ implStatFS
+
+ locks vfs.FileLocks
+}
+
+// Keep implements kernfs.Inode.Keep.
+func (*dir) Keep() bool {
+ return true
+}
+
+// SetStat implements kernfs.Inode.SetStat not allowing inode attributes to be changed.
+func (*dir) SetStat(context.Context, *vfs.Filesystem, *auth.Credentials, vfs.SetStatOptions) error {
+ return syserror.EPERM
+}
+
+// Open implements kernfs.Inode.Open.
+func (d *dir) Open(ctx context.Context, rp *vfs.ResolvingPath, kd *kernfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+ fd, err := kernfs.NewGenericDirectoryFD(rp.Mount(), kd, &d.OrderedChildren, &d.locks, &opts, kernfs.GenericDirectoryFDOptions{
+ SeekEnd: kernfs.SeekEndStaticEntries,
+ })
+ if err != nil {
+ return nil, err
+ }
+ return fd.VFSFileDescription(), nil
+}
+
+// DecRef implements kernfs.Inode.DecRef.
+func (d *dir) DecRef(ctx context.Context) {
+ d.dirRefs.DecRef(func() { d.Destroy(ctx) })
+}
+
+// StatFS implements kernfs.Inode.StatFS.
+func (d *dir) StatFS(ctx context.Context, fs *vfs.Filesystem) (linux.Statfs, error) {
+ return vfs.GenericStatFS(linux.CGROUP_SUPER_MAGIC), nil
+}
+
+// controllerFile represents a generic control file that appears within a cgroup
+// directory.
+//
+// +stateify savable
+type controllerFile struct {
+ kernfs.DynamicBytesFile
+}
+
+func (fs *filesystem) newControllerFile(ctx context.Context, creds *auth.Credentials, data vfs.DynamicBytesSource) kernfs.Inode {
+ f := &controllerFile{}
+ f.Init(ctx, creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), data, readonlyFileMode)
+ return f
+}
+
+func (fs *filesystem) newControllerWritableFile(ctx context.Context, creds *auth.Credentials, data vfs.WritableDynamicBytesSource) kernfs.Inode {
+ f := &controllerFile{}
+ f.Init(ctx, creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), data, writableFileMode)
+ return f
+}
+
+// staticControllerFile represents a generic control file that appears within a
+// cgroup directory which always returns the same data when read.
+// staticControllerFiles are not writable.
+//
+// +stateify savable
+type staticControllerFile struct {
+ kernfs.DynamicBytesFile
+ vfs.StaticData
+}
+
+// Note: We let the caller provide the mode so that static files may be used to
+// fake both readable and writable control files. However, static files are
+// effectively readonly, as attempting to write to them will return EIO
+// regardless of the mode.
+func (fs *filesystem) newStaticControllerFile(ctx context.Context, creds *auth.Credentials, mode linux.FileMode, data string) kernfs.Inode {
+ f := &staticControllerFile{StaticData: vfs.StaticData{Data: data}}
+ f.Init(ctx, creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), f, mode)
+ return f
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/cpu.go b/pkg/sentry/fsimpl/cgroupfs/cpu.go
new file mode 100644
index 000000000..24d86a277
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/cpu.go
@@ -0,0 +1,70 @@
+// Copyright 2021 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 cgroupfs
+
+import (
+ "fmt"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// +stateify savable
+type cpuController struct {
+ controllerCommon
+
+ // CFS bandwidth control parameters, values in microseconds.
+ cfsPeriod int64
+ cfsQuota int64
+
+ // CPU shares, values should be (num core * 1024).
+ shares int64
+}
+
+var _ controller = (*cpuController)(nil)
+
+func newCPUController(fs *filesystem, defaults map[string]int64) *cpuController {
+ // Default values for controller parameters from Linux.
+ c := &cpuController{
+ cfsPeriod: 100000,
+ cfsQuota: -1,
+ shares: 1024,
+ }
+
+ if val, ok := defaults["cpu.cfs_period_us"]; ok {
+ c.cfsPeriod = val
+ delete(defaults, "cpu.cfs_period_us")
+ }
+ if val, ok := defaults["cpu.cfs_quota_us"]; ok {
+ c.cfsQuota = val
+ delete(defaults, "cpu.cfs_quota_us")
+ }
+ if val, ok := defaults["cpu.shares"]; ok {
+ c.shares = val
+ delete(defaults, "cpu.shares")
+ }
+
+ c.controllerCommon.init(controllerCPU, fs)
+ return c
+}
+
+// AddControlFiles implements controller.AddControlFiles.
+func (c *cpuController) AddControlFiles(ctx context.Context, creds *auth.Credentials, _ *cgroupInode, contents map[string]kernfs.Inode) {
+ contents["cpu.cfs_period_us"] = c.fs.newStaticControllerFile(ctx, creds, linux.FileMode(0644), fmt.Sprintf("%d\n", c.cfsPeriod))
+ contents["cpu.cfs_quota_us"] = c.fs.newStaticControllerFile(ctx, creds, linux.FileMode(0644), fmt.Sprintf("%d\n", c.cfsQuota))
+ contents["cpu.shares"] = c.fs.newStaticControllerFile(ctx, creds, linux.FileMode(0644), fmt.Sprintf("%d\n", c.shares))
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/cpuacct.go b/pkg/sentry/fsimpl/cgroupfs/cpuacct.go
new file mode 100644
index 000000000..d4104a00e
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/cpuacct.go
@@ -0,0 +1,114 @@
+// Copyright 2021 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 cgroupfs
+
+import (
+ "bytes"
+ "fmt"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/usage"
+)
+
+// +stateify savable
+type cpuacctController struct {
+ controllerCommon
+}
+
+var _ controller = (*cpuacctController)(nil)
+
+func newCPUAcctController(fs *filesystem) *cpuacctController {
+ c := &cpuacctController{}
+ c.controllerCommon.init(controllerCPUAcct, fs)
+ return c
+}
+
+// AddControlFiles implements controller.AddControlFiles.
+func (c *cpuacctController) AddControlFiles(ctx context.Context, creds *auth.Credentials, cg *cgroupInode, contents map[string]kernfs.Inode) {
+ cpuacctCG := &cpuacctCgroup{cg}
+ contents["cpuacct.stat"] = c.fs.newControllerFile(ctx, creds, &cpuacctStatData{cpuacctCG})
+ contents["cpuacct.usage"] = c.fs.newControllerFile(ctx, creds, &cpuacctUsageData{cpuacctCG})
+ contents["cpuacct.usage_user"] = c.fs.newControllerFile(ctx, creds, &cpuacctUsageUserData{cpuacctCG})
+ contents["cpuacct.usage_sys"] = c.fs.newControllerFile(ctx, creds, &cpuacctUsageSysData{cpuacctCG})
+}
+
+// +stateify savable
+type cpuacctCgroup struct {
+ *cgroupInode
+}
+
+func (c *cpuacctCgroup) collectCPUStats() usage.CPUStats {
+ var cs usage.CPUStats
+ c.fs.tasksMu.RLock()
+ // Note: This isn't very accurate, since the tasks are potentially
+ // still running as we accumulate their stats.
+ for t := range c.ts {
+ cs.Accumulate(t.CPUStats())
+ }
+ c.fs.tasksMu.RUnlock()
+ return cs
+}
+
+// +stateify savable
+type cpuacctStatData struct {
+ *cpuacctCgroup
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *cpuacctStatData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ cs := d.collectCPUStats()
+ fmt.Fprintf(buf, "user %d\n", linux.ClockTFromDuration(cs.UserTime))
+ fmt.Fprintf(buf, "system %d\n", linux.ClockTFromDuration(cs.SysTime))
+ return nil
+}
+
+// +stateify savable
+type cpuacctUsageData struct {
+ *cpuacctCgroup
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *cpuacctUsageData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ cs := d.collectCPUStats()
+ fmt.Fprintf(buf, "%d\n", cs.UserTime.Nanoseconds()+cs.SysTime.Nanoseconds())
+ return nil
+}
+
+// +stateify savable
+type cpuacctUsageUserData struct {
+ *cpuacctCgroup
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *cpuacctUsageUserData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ cs := d.collectCPUStats()
+ fmt.Fprintf(buf, "%d\n", cs.UserTime.Nanoseconds())
+ return nil
+}
+
+// +stateify savable
+type cpuacctUsageSysData struct {
+ *cpuacctCgroup
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *cpuacctUsageSysData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ cs := d.collectCPUStats()
+ fmt.Fprintf(buf, "%d\n", cs.SysTime.Nanoseconds())
+ return nil
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/cpuset.go b/pkg/sentry/fsimpl/cgroupfs/cpuset.go
new file mode 100644
index 000000000..ac547f8e2
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/cpuset.go
@@ -0,0 +1,39 @@
+// Copyright 2021 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 cgroupfs
+
+import (
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// +stateify savable
+type cpusetController struct {
+ controllerCommon
+}
+
+var _ controller = (*cpusetController)(nil)
+
+func newCPUSetController(fs *filesystem) *cpusetController {
+ c := &cpusetController{}
+ c.controllerCommon.init(controllerCPUSet, fs)
+ return c
+}
+
+// AddControlFiles implements controller.AddControlFiles.
+func (c *cpusetController) AddControlFiles(ctx context.Context, creds *auth.Credentials, _ *cgroupInode, contents map[string]kernfs.Inode) {
+ // This controller is currently intentionally empty.
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/job.go b/pkg/sentry/fsimpl/cgroupfs/job.go
new file mode 100644
index 000000000..48919c338
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/job.go
@@ -0,0 +1,64 @@
+// Copyright 2021 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 cgroupfs
+
+import (
+ "bytes"
+ "fmt"
+
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+// +stateify savable
+type jobController struct {
+ controllerCommon
+ id int64
+}
+
+var _ controller = (*jobController)(nil)
+
+func newJobController(fs *filesystem) *jobController {
+ c := &jobController{}
+ c.controllerCommon.init(controllerJob, fs)
+ return c
+}
+
+func (c *jobController) AddControlFiles(ctx context.Context, creds *auth.Credentials, _ *cgroupInode, contents map[string]kernfs.Inode) {
+ contents["job.id"] = c.fs.newControllerWritableFile(ctx, creds, &jobIDData{c: c})
+}
+
+// +stateify savable
+type jobIDData struct {
+ c *jobController
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *jobIDData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ fmt.Fprintf(buf, "%d\n", d.c.id)
+ return nil
+}
+
+// Write implements vfs.WritableDynamicBytesSource.Write.
+func (d *jobIDData) Write(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
+ val, n, err := parseInt64FromString(ctx, src, offset)
+ if err != nil {
+ return n, err
+ }
+ d.c.id = val
+ return n, nil
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/memory.go b/pkg/sentry/fsimpl/cgroupfs/memory.go
new file mode 100644
index 000000000..485c98376
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/memory.go
@@ -0,0 +1,74 @@
+// Copyright 2021 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 cgroupfs
+
+import (
+ "bytes"
+ "fmt"
+ "math"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/usage"
+)
+
+// +stateify savable
+type memoryController struct {
+ controllerCommon
+
+ limitBytes int64
+}
+
+var _ controller = (*memoryController)(nil)
+
+func newMemoryController(fs *filesystem, defaults map[string]int64) *memoryController {
+ c := &memoryController{
+ // Linux sets this to (PAGE_COUNTER_MAX * PAGE_SIZE) by default, which
+ // is ~ 2**63 on a 64-bit system. So essentially, inifinity. The exact
+ // value isn't very important.
+ limitBytes: math.MaxInt64,
+ }
+ if val, ok := defaults["memory.limit_in_bytes"]; ok {
+ c.limitBytes = val
+ delete(defaults, "memory.limit_in_bytes")
+ }
+ c.controllerCommon.init(controllerMemory, fs)
+ return c
+}
+
+// AddControlFiles implements controller.AddControlFiles.
+func (c *memoryController) AddControlFiles(ctx context.Context, creds *auth.Credentials, _ *cgroupInode, contents map[string]kernfs.Inode) {
+ contents["memory.usage_in_bytes"] = c.fs.newControllerFile(ctx, creds, &memoryUsageInBytesData{})
+ contents["memory.limit_in_bytes"] = c.fs.newStaticControllerFile(ctx, creds, linux.FileMode(0644), fmt.Sprintf("%d\n", c.limitBytes))
+}
+
+// +stateify savable
+type memoryUsageInBytesData struct{}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *memoryUsageInBytesData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ // TODO(b/183151557): This is a giant hack, we're using system-wide
+ // accounting since we know there is only one cgroup.
+ k := kernel.KernelFromContext(ctx)
+ mf := k.MemoryFile()
+ mf.UpdateUsage()
+ _, totalBytes := usage.MemoryAccounting.Copy()
+
+ fmt.Fprintf(buf, "%d\n", totalBytes)
+ return nil
+}
diff --git a/pkg/sentry/fsimpl/eventfd/BUILD b/pkg/sentry/fsimpl/eventfd/BUILD
index bcb01bb08..c09fdc7f9 100644
--- a/pkg/sentry/fsimpl/eventfd/BUILD
+++ b/pkg/sentry/fsimpl/eventfd/BUILD
@@ -10,6 +10,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/fdnotifier",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/sentry/vfs",
"//pkg/syserror",
diff --git a/pkg/sentry/fsimpl/eventfd/eventfd.go b/pkg/sentry/fsimpl/eventfd/eventfd.go
index 30bd05357..4f79cfcb7 100644
--- a/pkg/sentry/fsimpl/eventfd/eventfd.go
+++ b/pkg/sentry/fsimpl/eventfd/eventfd.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fdnotifier"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
@@ -188,7 +189,7 @@ func (efd *EventFileDescription) read(ctx context.Context, dst usermem.IOSequenc
efd.queue.Notify(waiter.WritableEvents)
var buf [8]byte
- usermem.ByteOrder.PutUint64(buf[:], val)
+ hostarch.ByteOrder.PutUint64(buf[:], val)
_, err := dst.CopyOut(ctx, buf[:])
return err
}
@@ -196,7 +197,7 @@ func (efd *EventFileDescription) read(ctx context.Context, dst usermem.IOSequenc
// Preconditions: Must be called with efd.mu locked.
func (efd *EventFileDescription) hostWriteLocked(val uint64) error {
var buf [8]byte
- usermem.ByteOrder.PutUint64(buf[:], val)
+ hostarch.ByteOrder.PutUint64(buf[:], val)
_, err := unix.Write(efd.hostfd, buf[:])
if err == unix.EWOULDBLOCK {
return syserror.ErrWouldBlock
@@ -209,7 +210,7 @@ func (efd *EventFileDescription) write(ctx context.Context, src usermem.IOSequen
if _, err := src.CopyIn(ctx, buf[:]); err != nil {
return err
}
- val := usermem.ByteOrder.Uint64(buf[:])
+ val := hostarch.ByteOrder.Uint64(buf[:])
return efd.Signal(val)
}
diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD
index 155c0f56d..3a4777fbe 100644
--- a/pkg/sentry/fsimpl/fuse/BUILD
+++ b/pkg/sentry/fsimpl/fuse/BUILD
@@ -46,6 +46,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/marshal",
"//pkg/refs",
@@ -75,6 +76,7 @@ go_test(
library = ":fuse",
deps = [
"//pkg/abi/linux",
+ "//pkg/hostarch",
"//pkg/marshal",
"//pkg/sentry/fsimpl/testutil",
"//pkg/sentry/kernel",
diff --git a/pkg/sentry/fsimpl/fuse/read_write.go b/pkg/sentry/fsimpl/fuse/read_write.go
index 23ce91849..66ea889f9 100644
--- a/pkg/sentry/fsimpl/fuse/read_write.go
+++ b/pkg/sentry/fsimpl/fuse/read_write.go
@@ -20,11 +20,11 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
// ReadInPages sends FUSE_READ requests for the size after round it up to
@@ -43,10 +43,10 @@ func (fs *filesystem) ReadInPages(ctx context.Context, fd *regularFileFD, off ui
}
// Round up to a multiple of page size.
- readSize, _ := usermem.PageRoundUp(uint64(size))
+ readSize, _ := hostarch.PageRoundUp(uint64(size))
// One request cannnot exceed either maxRead or maxPages.
- maxPages := fs.conn.maxRead >> usermem.PageShift
+ maxPages := fs.conn.maxRead >> hostarch.PageShift
if maxPages > uint32(fs.conn.maxPages) {
maxPages = uint32(fs.conn.maxPages)
}
@@ -54,9 +54,9 @@ func (fs *filesystem) ReadInPages(ctx context.Context, fd *regularFileFD, off ui
var outs [][]byte
var sizeRead uint32
- // readSize is a multiple of usermem.PageSize.
+ // readSize is a multiple of hostarch.PageSize.
// Always request bytes as a multiple of pages.
- pagesRead, pagesToRead := uint32(0), uint32(readSize>>usermem.PageShift)
+ pagesRead, pagesToRead := uint32(0), uint32(readSize>>hostarch.PageShift)
// Reuse the same struct for unmarshalling to avoid unnecessary memory allocation.
in := linux.FUSEReadIn{
@@ -76,8 +76,8 @@ func (fs *filesystem) ReadInPages(ctx context.Context, fd *regularFileFD, off ui
pagesCanRead = maxPages
}
- in.Offset = off + (uint64(pagesRead) << usermem.PageShift)
- in.Size = pagesCanRead << usermem.PageShift
+ in.Offset = off + (uint64(pagesRead) << hostarch.PageShift)
+ in.Size = pagesCanRead << hostarch.PageShift
// TODO(gvisor.dev/issue/3247): support async read.
@@ -159,7 +159,7 @@ func (fs *filesystem) Write(ctx context.Context, fd *regularFileFD, off uint64,
}
// One request cannnot exceed either maxWrite or maxPages.
- maxWrite := uint32(fs.conn.maxPages) << usermem.PageShift
+ maxWrite := uint32(fs.conn.maxPages) << hostarch.PageShift
if maxWrite > fs.conn.maxWrite {
maxWrite = fs.conn.maxWrite
}
@@ -188,8 +188,8 @@ func (fs *filesystem) Write(ctx context.Context, fd *regularFileFD, off uint64,
// Limit the write size to one page.
// Note that the bigWrites flag is obsolete,
// latest libfuse always sets it on.
- if !fs.conn.bigWrites && toWrite > usermem.PageSize {
- toWrite = usermem.PageSize
+ if !fs.conn.bigWrites && toWrite > hostarch.PageSize {
+ toWrite = hostarch.PageSize
}
// Limit the write size to maxWrite.
diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go
index 10fb9d7d2..8a72489fa 100644
--- a/pkg/sentry/fsimpl/fuse/request_response.go
+++ b/pkg/sentry/fsimpl/fuse/request_response.go
@@ -19,10 +19,10 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
- "gvisor.dev/gvisor/pkg/usermem"
)
// fuseInitRes is a variable-length wrapper of linux.FUSEInitOut. The FUSE
@@ -45,29 +45,29 @@ func (r *fuseInitRes) UnmarshalBytes(src []byte) {
out := &r.initOut
// Introduced before FUSE kernel version 7.13.
- out.Major = uint32(usermem.ByteOrder.Uint32(src[:4]))
+ out.Major = uint32(hostarch.ByteOrder.Uint32(src[:4]))
src = src[4:]
- out.Minor = uint32(usermem.ByteOrder.Uint32(src[:4]))
+ out.Minor = uint32(hostarch.ByteOrder.Uint32(src[:4]))
src = src[4:]
- out.MaxReadahead = uint32(usermem.ByteOrder.Uint32(src[:4]))
+ out.MaxReadahead = uint32(hostarch.ByteOrder.Uint32(src[:4]))
src = src[4:]
- out.Flags = uint32(usermem.ByteOrder.Uint32(src[:4]))
+ out.Flags = uint32(hostarch.ByteOrder.Uint32(src[:4]))
src = src[4:]
- out.MaxBackground = uint16(usermem.ByteOrder.Uint16(src[:2]))
+ out.MaxBackground = uint16(hostarch.ByteOrder.Uint16(src[:2]))
src = src[2:]
- out.CongestionThreshold = uint16(usermem.ByteOrder.Uint16(src[:2]))
+ out.CongestionThreshold = uint16(hostarch.ByteOrder.Uint16(src[:2]))
src = src[2:]
- out.MaxWrite = uint32(usermem.ByteOrder.Uint32(src[:4]))
+ out.MaxWrite = uint32(hostarch.ByteOrder.Uint32(src[:4]))
src = src[4:]
// Introduced in FUSE kernel version 7.23.
if len(src) >= 4 {
- out.TimeGran = uint32(usermem.ByteOrder.Uint32(src[:4]))
+ out.TimeGran = uint32(hostarch.ByteOrder.Uint32(src[:4]))
src = src[4:]
}
// Introduced in FUSE kernel version 7.28.
if len(src) >= 2 {
- out.MaxPages = uint16(usermem.ByteOrder.Uint16(src[:2]))
+ out.MaxPages = uint16(hostarch.ByteOrder.Uint16(src[:2]))
src = src[2:]
}
_ = src // Remove unused warning.
diff --git a/pkg/sentry/fsimpl/fuse/utils_test.go b/pkg/sentry/fsimpl/fuse/utils_test.go
index 2c0cc0f4e..b0bab0066 100644
--- a/pkg/sentry/fsimpl/fuse/utils_test.go
+++ b/pkg/sentry/fsimpl/fuse/utils_test.go
@@ -24,7 +24,8 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
- "gvisor.dev/gvisor/pkg/usermem"
+
+ "gvisor.dev/gvisor/pkg/hostarch"
)
func setup(t *testing.T) *testutil.System {
@@ -82,12 +83,12 @@ func (t *testPayload) SizeBytes() int {
// MarshalBytes implements marshal.Marshallable.MarshalBytes.
func (t *testPayload) MarshalBytes(dst []byte) {
- usermem.ByteOrder.PutUint32(dst[:4], t.data)
+ hostarch.ByteOrder.PutUint32(dst[:4], t.data)
}
// UnmarshalBytes implements marshal.Marshallable.UnmarshalBytes.
func (t *testPayload) UnmarshalBytes(src []byte) {
- *t = testPayload{data: usermem.ByteOrder.Uint32(src[:4])}
+ *t = testPayload{data: hostarch.ByteOrder.Uint32(src[:4])}
}
// Packed implements marshal.Marshallable.Packed.
@@ -106,17 +107,17 @@ func (t *testPayload) UnmarshalUnsafe(src []byte) {
}
// CopyOutN implements marshal.Marshallable.CopyOutN.
-func (t *testPayload) CopyOutN(task marshal.CopyContext, addr usermem.Addr, limit int) (int, error) {
+func (t *testPayload) CopyOutN(task marshal.CopyContext, addr hostarch.Addr, limit int) (int, error) {
panic("not implemented")
}
// CopyOut implements marshal.Marshallable.CopyOut.
-func (t *testPayload) CopyOut(task marshal.CopyContext, addr usermem.Addr) (int, error) {
+func (t *testPayload) CopyOut(task marshal.CopyContext, addr hostarch.Addr) (int, error) {
panic("not implemented")
}
// CopyIn implements marshal.Marshallable.CopyIn.
-func (t *testPayload) CopyIn(task marshal.CopyContext, addr usermem.Addr) (int, error) {
+func (t *testPayload) CopyIn(task marshal.CopyContext, addr hostarch.Addr) (int, error) {
panic("not implemented")
}
diff --git a/pkg/sentry/fsimpl/gofer/BUILD b/pkg/sentry/fsimpl/gofer/BUILD
index 807b6ed1f..6d5258a9b 100644
--- a/pkg/sentry/fsimpl/gofer/BUILD
+++ b/pkg/sentry/fsimpl/gofer/BUILD
@@ -51,6 +51,7 @@ go_library(
"//pkg/fd",
"//pkg/fdnotifier",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/p9",
"//pkg/refs",
diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go
index 9da01cba3..177e42649 100644
--- a/pkg/sentry/fsimpl/gofer/directory.go
+++ b/pkg/sentry/fsimpl/gofer/directory.go
@@ -20,6 +20,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -28,7 +29,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
func (d *dentry) isDir() bool {
@@ -98,7 +98,7 @@ func (d *dentry) createSyntheticChildLocked(opts *createSyntheticOpts) {
mode: uint32(opts.mode),
uid: uint32(opts.kuid),
gid: uint32(opts.kgid),
- blockSize: usermem.PageSize, // arbitrary
+ blockSize: hostarch.PageSize, // arbitrary
atime: now,
mtime: now,
ctime: now,
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 43c3c5a2d..4b5621043 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -141,21 +141,8 @@ func (fs *filesystem) renameMuRUnlockAndCheckCaching(ctx context.Context, dsp **
return
}
ds := **dsp
- // Only go through calling dentry.checkCachingLocked() (which requires
- // re-locking renameMu) if we actually have any dentries with zero refs.
- checkAny := false
- for i := range ds {
- if atomic.LoadInt64(&ds[i].refs) == 0 {
- checkAny = true
- break
- }
- }
- if checkAny {
- fs.renameMu.Lock()
- for _, d := range ds {
- d.checkCachingLocked(ctx)
- }
- fs.renameMu.Unlock()
+ for _, d := range ds {
+ d.checkCachingLocked(ctx, false /* renameMuWriteLocked */)
}
putDentrySlice(*dsp)
}
@@ -166,7 +153,7 @@ func (fs *filesystem) renameMuUnlockAndCheckCaching(ctx context.Context, ds **[]
return
}
for _, d := range **ds {
- d.checkCachingLocked(ctx)
+ d.checkCachingLocked(ctx, true /* renameMuWriteLocked */)
}
fs.renameMu.Unlock()
putDentrySlice(*ds)
@@ -339,8 +326,10 @@ func (fs *filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.Vir
}
parent.cacheNewChildLocked(child, name)
// For now, child has 0 references, so our caller should call
- // child.checkCachingLocked().
+ // child.checkCachingLocked(). parent gained a ref so we should also call
+ // parent.checkCachingLocked() so it can be removed from the cache if needed.
*ds = appendDentry(*ds, child)
+ *ds = appendDentry(*ds, parent)
return child, nil
}
@@ -723,6 +712,8 @@ func (fs *filesystem) GetDentryAt(ctx context.Context, rp *vfs.ResolvingPath, op
}
}
d.IncRef()
+ // Call d.checkCachingLocked() so it can be removed from the cache if needed.
+ ds = appendDentry(ds, d)
return &d.vfsd, nil
}
@@ -744,6 +735,8 @@ func (fs *filesystem) GetParentDentryAt(ctx context.Context, rp *vfs.ResolvingPa
return nil, err
}
d.IncRef()
+ // Call d.checkCachingLocked() so it can be removed from the cache if needed.
+ ds = appendDentry(ds, d)
return &d.vfsd, nil
}
@@ -782,7 +775,7 @@ func (fs *filesystem) LinkAt(ctx context.Context, rp *vfs.ResolvingPath, vd vfs.
// MkdirAt implements vfs.FilesystemImpl.MkdirAt.
func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.MkdirOptions) error {
creds := rp.Credentials()
- return fs.doCreateAt(ctx, rp, true /* dir */, func(parent *dentry, name string, _ **[]*dentry) error {
+ return fs.doCreateAt(ctx, rp, true /* dir */, func(parent *dentry, name string, ds **[]*dentry) error {
// If the parent is a setgid directory, use the parent's GID
// rather than the caller's and enable setgid.
kgid := creds.EffectiveKGID
@@ -802,6 +795,7 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
kuid: creds.EffectiveKUID,
kgid: creds.EffectiveKGID,
})
+ *ds = appendDentry(*ds, parent)
}
if fs.opts.interop != InteropModeShared {
parent.incLinks()
@@ -855,6 +849,7 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
kgid: creds.EffectiveKGID,
endpoint: opts.Endpoint,
})
+ *ds = appendDentry(*ds, parent)
return nil
case linux.S_IFIFO:
parent.createSyntheticChildLocked(&createSyntheticOpts{
@@ -864,6 +859,7 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
kgid: creds.EffectiveKGID,
pipe: pipe.NewVFSPipe(true /* isNamed */, pipe.DefaultPipeSize),
})
+ *ds = appendDentry(*ds, parent)
return nil
}
// Retain error from gofer if synthetic file cannot be created internally.
@@ -912,6 +908,8 @@ func (fs *filesystem) OpenAt(ctx context.Context, rp *vfs.ResolvingPath, opts vf
start.IncRef()
defer start.DecRef(ctx)
unlock()
+ // start is intentionally not added to ds (which would remove it from the
+ // cache) because doing so regresses performance in practice.
return start.open(ctx, rp, &opts)
}
@@ -965,6 +963,8 @@ afterTrailingSymlink:
child.IncRef()
defer child.DecRef(ctx)
unlock()
+ // child is intentionally not added to ds (which would remove it from the
+ // cache) because doing so regresses performance in practice.
return child.open(ctx, rp, &opts)
}
@@ -1212,6 +1212,7 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
}
// Insert the dentry into the tree.
d.cacheNewChildLocked(child, name)
+ *ds = appendDentry(*ds, d)
if d.cachedMetadataAuthoritative() {
d.touchCMtime()
d.dirents = nil
@@ -1403,6 +1404,7 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
oldParent.decRefNoCaching()
ds = appendDentry(ds, oldParent)
newParent.IncRef()
+ ds = appendDentry(ds, newParent)
if renamed.isSynthetic() {
oldParent.syntheticChildren--
newParent.syntheticChildren++
@@ -1546,6 +1548,7 @@ func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath
if d.isSocket() {
if !d.isSynthetic() {
d.IncRef()
+ ds = appendDentry(ds, d)
return &endpoint{
dentry: d,
path: opts.Addr,
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index 692da02c1..fb42c5f62 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -18,15 +18,17 @@
// Lock order:
// regularFileFD/directoryFD.mu
// filesystem.renameMu
-// dentry.dirMu
-// filesystem.syncMu
-// dentry.metadataMu
-// *** "memmap.Mappable locks" below this point
-// dentry.mapsMu
-// *** "memmap.Mappable locks taken by Translate" below this point
-// dentry.handleMu
-// dentry.dataMu
-// filesystem.inoMu
+// dentry.cachingMu
+// filesystem.cacheMu
+// dentry.dirMu
+// filesystem.syncMu
+// dentry.metadataMu
+// *** "memmap.Mappable locks" below this point
+// dentry.mapsMu
+// *** "memmap.Mappable locks taken by Translate" below this point
+// dentry.handleMu
+// dentry.dataMu
+// filesystem.inoMu
// specialFileFD.mu
// specialFileFD.bufMu
//
@@ -44,6 +46,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/p9"
refs_vfs1 "gvisor.dev/gvisor/pkg/refs"
@@ -60,7 +63,6 @@ import (
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/unet"
- "gvisor.dev/gvisor/pkg/usermem"
)
// Name is the default filesystem name.
@@ -140,7 +142,8 @@ type filesystem struct {
// cachedDentries contains all dentries with 0 references. (Due to race
// conditions, it may also contain dentries with non-zero references.)
// cachedDentriesLen is the number of dentries in cachedDentries. These fields
- // are protected by renameMu.
+ // are protected by cacheMu.
+ cacheMu sync.Mutex `state:"nosave"`
cachedDentries dentryList
cachedDentriesLen uint64
@@ -620,11 +623,11 @@ func (fs *filesystem) Release(ctx context.Context) {
// the reference count on every synthetic dentry. Synthetic dentries have one
// reference for existence that should be dropped during filesystem.Release.
//
-// Precondition: d.fs.renameMu is locked.
+// Precondition: d.fs.renameMu is locked for writing.
func (d *dentry) releaseSyntheticRecursiveLocked(ctx context.Context) {
if d.isSynthetic() {
d.decRefNoCaching()
- d.checkCachingLocked(ctx)
+ d.checkCachingLocked(ctx, true /* renameMuWriteLocked */)
}
if d.isDir() {
var children []*dentry
@@ -682,9 +685,13 @@ type dentry struct {
// deleted. deleted is accessed using atomic memory operations.
deleted uint32
+ // cachingMu is used to synchronize concurrent dentry caching attempts on
+ // this dentry.
+ cachingMu sync.Mutex `state:"nosave"`
+
// If cached is true, dentryEntry links dentry into
// filesystem.cachedDentries. cached and dentryEntry are protected by
- // filesystem.renameMu.
+ // cachingMu.
cached bool
dentryEntry
@@ -872,7 +879,7 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma
mode: uint32(attr.Mode),
uid: uint32(fs.opts.dfltuid),
gid: uint32(fs.opts.dfltgid),
- blockSize: usermem.PageSize,
+ blockSize: hostarch.PageSize,
readFD: -1,
writeFD: -1,
mmapFD: -1,
@@ -980,36 +987,63 @@ func (d *dentry) updateFromP9AttrsLocked(mask p9.AttrMask, attr *p9.Attr) {
}
// Preconditions: !d.isSynthetic().
+// Preconditions: d.metadataMu is locked.
+func (d *dentry) refreshSizeLocked(ctx context.Context) error {
+ d.handleMu.RLock()
+
+ if d.writeFD < 0 {
+ d.handleMu.RUnlock()
+ // Ask the gofer if we don't have a host FD.
+ return d.updateFromGetattrLocked(ctx)
+ }
+
+ var stat unix.Statx_t
+ err := unix.Statx(int(d.writeFD), "", unix.AT_EMPTY_PATH, unix.STATX_SIZE, &stat)
+ d.handleMu.RUnlock() // must be released before updateSizeLocked()
+ if err != nil {
+ return err
+ }
+ d.updateSizeLocked(stat.Size)
+ return nil
+}
+
+// Preconditions: !d.isSynthetic().
func (d *dentry) updateFromGetattr(ctx context.Context) error {
- // Use d.readFile or d.writeFile, which represent 9P fids that have been
+ // d.metadataMu must be locked *before* we getAttr so that we do not end up
+ // updating stale attributes in d.updateFromP9AttrsLocked().
+ d.metadataMu.Lock()
+ defer d.metadataMu.Unlock()
+ return d.updateFromGetattrLocked(ctx)
+}
+
+// Preconditions:
+// * !d.isSynthetic().
+// * d.metadataMu is locked.
+func (d *dentry) updateFromGetattrLocked(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.
// 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
- )
- // d.metadataMu must be locked *before* we getAttr so that we do not end up
- // updating stale attributes in d.updateFromP9AttrsLocked().
- d.metadataMu.Lock()
- defer d.metadataMu.Unlock()
d.handleMu.RLock()
- if !d.writeFile.isNil() {
+ handleMuRLocked := true
+ var file p9file
+ switch {
+ case !d.writeFile.isNil():
file = d.writeFile
- handleMuRLocked = true
- } else if !d.readFile.isNil() {
+ case !d.readFile.isNil():
file = d.readFile
- handleMuRLocked = true
- } else {
+ default:
file = d.file
d.handleMu.RUnlock()
+ handleMuRLocked = false
}
+
_, attrMask, attr, err := file.getAttr(ctx, dentryAttrMask())
if handleMuRLocked {
- d.handleMu.RUnlock()
+ d.handleMu.RUnlock() // must be released before updateFromP9AttrsLocked()
}
if err != nil {
return err
@@ -1104,24 +1138,27 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
defer d.metadataMu.Unlock()
// As with Linux, if the UID, GID, or file size is changing, we have to
- // clear permission bits. Note that when set, clearSGID causes
- // permissions to be updated, but does not modify stat.Mask, as
- // modification would cause an extra inotify flag to be set.
- clearSGID := stat.Mask&linux.STATX_UID != 0 && stat.UID != atomic.LoadUint32(&d.uid) ||
- stat.Mask&linux.STATX_GID != 0 && stat.GID != atomic.LoadUint32(&d.gid) ||
+ // clear permission bits. Note that when set, clearSGID may cause
+ // permissions to be updated.
+ clearSGID := (stat.Mask&linux.STATX_UID != 0 && stat.UID != atomic.LoadUint32(&d.uid)) ||
+ (stat.Mask&linux.STATX_GID != 0 && stat.GID != atomic.LoadUint32(&d.gid)) ||
stat.Mask&linux.STATX_SIZE != 0
if clearSGID {
if stat.Mask&linux.STATX_MODE != 0 {
stat.Mode = uint16(vfs.ClearSUIDAndSGID(uint32(stat.Mode)))
} else {
- stat.Mode = uint16(vfs.ClearSUIDAndSGID(atomic.LoadUint32(&d.mode)))
+ oldMode := atomic.LoadUint32(&d.mode)
+ if updatedMode := vfs.ClearSUIDAndSGID(oldMode); updatedMode != oldMode {
+ stat.Mode = uint16(updatedMode)
+ stat.Mask |= linux.STATX_MODE
+ }
}
}
if !d.isSynthetic() {
if stat.Mask != 0 {
if err := d.file.setAttr(ctx, p9.SetAttrMask{
- Permissions: stat.Mask&linux.STATX_MODE != 0 || clearSGID,
+ Permissions: stat.Mask&linux.STATX_MODE != 0,
UID: stat.Mask&linux.STATX_UID != 0,
GID: stat.Mask&linux.STATX_GID != 0,
Size: stat.Mask&linux.STATX_SIZE != 0,
@@ -1156,7 +1193,7 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
return nil
}
}
- if stat.Mask&linux.STATX_MODE != 0 || clearSGID {
+ if stat.Mask&linux.STATX_MODE != 0 {
atomic.StoreUint32(&d.mode, d.fileType()|uint32(stat.Mode))
}
if stat.Mask&linux.STATX_UID != 0 {
@@ -1217,8 +1254,8 @@ func (d *dentry) updateSizeLocked(newSize uint64) {
// so we can't race with Write or another truncate.)
d.dataMu.Unlock()
if d.size < oldSize {
- oldpgend, _ := usermem.PageRoundUp(oldSize)
- newpgend, _ := usermem.PageRoundUp(d.size)
+ oldpgend, _ := hostarch.PageRoundUp(oldSize)
+ newpgend, _ := hostarch.PageRoundUp(d.size)
if oldpgend != newpgend {
d.mapsMu.Lock()
d.mappings.Invalidate(memmap.MappableRange{newpgend, oldpgend}, memmap.InvalidateOpts{
@@ -1312,9 +1349,7 @@ func (d *dentry) TryIncRef() bool {
// DecRef implements vfs.DentryImpl.DecRef.
func (d *dentry) DecRef(ctx context.Context) {
if d.decRefNoCaching() == 0 {
- d.fs.renameMu.Lock()
- d.checkCachingLocked(ctx)
- d.fs.renameMu.Unlock()
+ d.checkCachingLocked(ctx, false /* renameMuWriteLocked */)
}
}
@@ -1374,15 +1409,16 @@ func (d *dentry) Watches() *vfs.Watches {
//
// If no watches are left on this dentry and it has no references, cache it.
func (d *dentry) OnZeroWatches(ctx context.Context) {
- if atomic.LoadInt64(&d.refs) == 0 {
- d.fs.renameMu.Lock()
- d.checkCachingLocked(ctx)
- d.fs.renameMu.Unlock()
- }
+ d.checkCachingLocked(ctx, false /* renameMuWriteLocked */)
}
-// checkCachingLocked should be called after d's reference count becomes 0 or it
-// becomes disowned.
+// checkCachingLocked should be called after d's reference count becomes 0 or
+// it becomes disowned.
+//
+// For performance, checkCachingLocked can also be called after d's reference
+// count becomes non-zero, so that d can be removed from the LRU cache. This
+// may help in reducing the size of the cache and hence reduce evictions. Note
+// that this is not necessary for correctness.
//
// It may be called on a destroyed dentry. For example,
// renameMu[R]UnlockAndCheckCaching may call checkCachingLocked multiple times
@@ -1390,33 +1426,46 @@ func (d *dentry) OnZeroWatches(ctx context.Context) {
// operation. One of the calls may destroy the dentry, so subsequent calls will
// do nothing.
//
-// Preconditions: d.fs.renameMu must be locked for writing; it may be
-// temporarily unlocked.
-func (d *dentry) checkCachingLocked(ctx context.Context) {
- // Dentries with a non-zero reference count must be retained. (The only way
- // to obtain a reference on a dentry with zero references is via path
- // resolution, which requires renameMu, so if d.refs is zero then it will
- // remain zero while we hold renameMu for writing.)
+// Preconditions: d.fs.renameMu must be locked for writing if
+// renameMuWriteLocked is true; it may be temporarily unlocked.
+func (d *dentry) checkCachingLocked(ctx context.Context, renameMuWriteLocked bool) {
+ d.cachingMu.Lock()
refs := atomic.LoadInt64(&d.refs)
if refs == -1 {
// Dentry has already been destroyed.
+ d.cachingMu.Unlock()
return
}
if refs > 0 {
- // This isn't strictly necessary (fs.cachedDentries is permitted to
- // contain dentries with non-zero refs, which are skipped by
- // fs.evictCachedDentryLocked() upon reaching the end of the LRU), but
- // since we are already holding fs.renameMu for writing we may as well.
+ // fs.cachedDentries is permitted to contain dentries with non-zero refs,
+ // which are skipped by fs.evictCachedDentryLocked() upon reaching the end
+ // of the LRU. But it is still beneficial to remove d from the cache as we
+ // are already holding d.cachingMu. Keeping a cleaner cache also reduces
+ // the number of evictions (which is expensive as it acquires fs.renameMu).
d.removeFromCacheLocked()
+ d.cachingMu.Unlock()
return
}
// Deleted and invalidated dentries with zero references are no longer
// reachable by path resolution and should be dropped immediately.
if d.vfsd.IsDead() {
+ d.removeFromCacheLocked()
+ d.cachingMu.Unlock()
+ if !renameMuWriteLocked {
+ // Need to lock d.fs.renameMu for writing as needed by d.destroyLocked().
+ d.fs.renameMu.Lock()
+ defer d.fs.renameMu.Unlock()
+ // Now that renameMu is locked for writing, no more refs can be taken on
+ // d because path resolution requires renameMu for reading at least.
+ if atomic.LoadInt64(&d.refs) != 0 {
+ // Destroy d only if its ref is still 0. If not, either someone took a
+ // ref on it or it got destroyed before fs.renameMu could be acquired.
+ return
+ }
+ }
if d.isDeleted() {
d.watches.HandleDeletion(ctx)
}
- d.removeFromCacheLocked()
d.destroyLocked(ctx)
return
}
@@ -1426,24 +1475,36 @@ func (d *dentry) checkCachingLocked(ctx context.Context) {
// d.watches cannot concurrently transition from zero to non-zero, because
// adding a watch requires holding a reference on d.
if d.watches.Size() > 0 {
- // As in the refs > 0 case, this is not strictly necessary.
+ // As in the refs > 0 case, removing d is beneficial.
d.removeFromCacheLocked()
+ d.cachingMu.Unlock()
return
}
if atomic.LoadInt32(&d.fs.released) != 0 {
+ d.cachingMu.Unlock()
+ if !renameMuWriteLocked {
+ // Need to lock d.fs.renameMu to access d.parent. Lock it for writing as
+ // needed by d.destroyLocked() later.
+ d.fs.renameMu.Lock()
+ defer d.fs.renameMu.Unlock()
+ }
if d.parent != nil {
d.parent.dirMu.Lock()
delete(d.parent.children, d.name)
d.parent.dirMu.Unlock()
}
d.destroyLocked(ctx)
+ return
}
+ d.fs.cacheMu.Lock()
// If d is already cached, just move it to the front of the LRU.
if d.cached {
d.fs.cachedDentries.Remove(d)
d.fs.cachedDentries.PushFront(d)
+ d.fs.cacheMu.Unlock()
+ d.cachingMu.Unlock()
return
}
// Cache the dentry, then evict the least recently used cached dentry if
@@ -1451,18 +1512,28 @@ func (d *dentry) checkCachingLocked(ctx context.Context) {
d.fs.cachedDentries.PushFront(d)
d.fs.cachedDentriesLen++
d.cached = true
- if d.fs.cachedDentriesLen > d.fs.opts.maxCachedDentries {
+ shouldEvict := d.fs.cachedDentriesLen > d.fs.opts.maxCachedDentries
+ d.fs.cacheMu.Unlock()
+ d.cachingMu.Unlock()
+
+ if shouldEvict {
+ if !renameMuWriteLocked {
+ // Need to lock d.fs.renameMu for writing as needed by
+ // d.evictCachedDentryLocked().
+ d.fs.renameMu.Lock()
+ defer d.fs.renameMu.Unlock()
+ }
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: d.fs.renameMu must be locked for writing.
+// Preconditions: d.cachingMu must be locked.
func (d *dentry) removeFromCacheLocked() {
if d.cached {
+ d.fs.cacheMu.Lock()
d.fs.cachedDentries.Remove(d)
d.fs.cachedDentriesLen--
+ d.fs.cacheMu.Unlock()
d.cached = false
}
}
@@ -1477,28 +1548,43 @@ func (fs *filesystem) evictAllCachedDentriesLocked(ctx context.Context) {
// Preconditions:
// * fs.renameMu must be locked for writing; it may be temporarily unlocked.
-// * fs.cachedDentriesLen != 0.
func (fs *filesystem) evictCachedDentryLocked(ctx context.Context) {
+ fs.cacheMu.Lock()
victim := fs.cachedDentries.Back()
+ fs.cacheMu.Unlock()
+ if victim == nil {
+ // fs.cachedDentries may have become empty between when it was checked and
+ // when we locked fs.cacheMu.
+ return
+ }
+
+ victim.cachingMu.Lock()
victim.removeFromCacheLocked()
// victim.refs or victim.watches.Size() may have become non-zero from an
// earlier path resolution since it was inserted into fs.cachedDentries.
- if atomic.LoadInt64(&victim.refs) == 0 && victim.watches.Size() == 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()
+ if atomic.LoadInt64(&victim.refs) != 0 || victim.watches.Size() != 0 {
+ victim.cachingMu.Unlock()
+ return
+ }
+ 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.destroyLocked(ctx)
+ victim.parent.dirMu.Unlock()
}
+ // Safe to unlock cachingMu now that victim.vfsd.IsDead(). Henceforth any
+ // concurrent caching attempts on victim will attempt to destroy it and so
+ // will try to acquire fs.renameMu (which we have already acquired). Hence,
+ // fs.renameMu will synchronize the destroy attempts.
+ victim.cachingMu.Unlock()
+ victim.destroyLocked(ctx)
}
// destroyLocked destroys the dentry.
@@ -1584,7 +1670,7 @@ func (d *dentry) destroyLocked(ctx context.Context) {
// Drop the reference held by d on its parent without recursively locking
// d.fs.renameMu.
if d.parent != nil && d.parent.decRefNoCaching() == 0 {
- d.parent.checkCachingLocked(ctx)
+ d.parent.checkCachingLocked(ctx, true /* renameMuWriteLocked */)
}
refsvfs2.Unregister(d)
}
diff --git a/pkg/sentry/fsimpl/gofer/gofer_test.go b/pkg/sentry/fsimpl/gofer/gofer_test.go
index 76f08e252..806392d50 100644
--- a/pkg/sentry/fsimpl/gofer/gofer_test.go
+++ b/pkg/sentry/fsimpl/gofer/gofer_test.go
@@ -55,7 +55,7 @@ func TestDestroyIdempotent(t *testing.T) {
fs.renameMu.Lock()
defer fs.renameMu.Unlock()
- child.checkCachingLocked(ctx)
+ child.checkCachingLocked(ctx, true /* renameMuWriteLocked */)
if got := atomic.LoadInt64(&child.refs); got != -1 {
t.Fatalf("child.refs=%d, want: -1", got)
}
@@ -63,6 +63,6 @@ func TestDestroyIdempotent(t *testing.T) {
if got := atomic.LoadInt64(&parent.refs); got != -1 {
t.Fatalf("parent.refs=%d, want: -1", got)
}
- child.checkCachingLocked(ctx)
- child.checkCachingLocked(ctx)
+ child.checkCachingLocked(ctx, true /* renameMuWriteLocked */)
+ child.checkCachingLocked(ctx, true /* renameMuWriteLocked */)
}
diff --git a/pkg/sentry/fsimpl/gofer/regular_file.go b/pkg/sentry/fsimpl/gofer/regular_file.go
index 4f1ad0c88..f0e7bbaf7 100644
--- a/pkg/sentry/fsimpl/gofer/regular_file.go
+++ b/pkg/sentry/fsimpl/gofer/regular_file.go
@@ -22,6 +22,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/safemem"
@@ -203,18 +204,19 @@ func (fd *regularFileFD) pwrite(ctx context.Context, src usermem.IOSequence, off
}
d := fd.dentry()
+
+ d.metadataMu.Lock()
+ defer d.metadataMu.Unlock()
+
// If the 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 {
+ if err := d.refreshSizeLocked(ctx); err != nil {
return 0, offset, err
}
}
- d.metadataMu.Lock()
- defer d.metadataMu.Unlock()
-
// Set offset to file size if the fd was opened with O_APPEND.
if fd.vfsfd.StatusFlags()&linux.O_APPEND != 0 {
// Holding d.metadataMu is sufficient for reading d.size.
@@ -291,8 +293,8 @@ func (fd *regularFileFD) writeCache(ctx context.Context, d *dentry, offset int64
}
// Remove touched pages from the cache.
- pgstart := usermem.PageRoundDown(uint64(offset))
- pgend, ok := usermem.PageRoundUp(uint64(offset + src.NumBytes()))
+ pgstart := hostarch.PageRoundDown(uint64(offset))
+ pgend, ok := hostarch.PageRoundUp(uint64(offset + src.NumBytes()))
if !ok {
return syserror.EINVAL
}
@@ -408,7 +410,7 @@ func (rw *dentryReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error)
switch {
case seg.Ok():
// Get internal mappings from the cache.
- ims, err := mf.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), usermem.Read)
+ ims, err := mf.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), hostarch.Read)
if err != nil {
dataMuUnlock()
rw.d.handleMu.RUnlock()
@@ -434,9 +436,9 @@ func (rw *dentryReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error)
if fillCache {
// Read into the cache, then re-enter the loop to read from the
// cache.
- gapEnd, _ := usermem.PageRoundUp(gapMR.End)
+ gapEnd, _ := hostarch.PageRoundUp(gapMR.End)
reqMR := memmap.MappableRange{
- Start: usermem.PageRoundDown(gapMR.Start),
+ Start: hostarch.PageRoundDown(gapMR.Start),
End: gapEnd,
}
optMR := gap.Range()
@@ -527,7 +529,7 @@ func (rw *dentryReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, erro
case seg.Ok():
// Get internal mappings from the cache.
segMR := seg.Range().Intersect(mr)
- ims, err := mf.MapInternal(seg.FileRangeOf(segMR), usermem.Write)
+ ims, err := mf.MapInternal(seg.FileRangeOf(segMR), hostarch.Write)
if err != nil {
retErr = err
goto exitLoop
@@ -700,6 +702,7 @@ func (fd *regularFileFD) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpt
}
// After this point, d may be used as a memmap.Mappable.
d.pf.hostFileMapperInitOnce.Do(d.pf.hostFileMapper.Init)
+ opts.SentryOwnedContent = d.fs.opts.forcePageCache
return vfs.GenericConfigureMMap(&fd.vfsfd, d, opts)
}
@@ -714,7 +717,7 @@ func (d *dentry) mayCachePages() bool {
}
// AddMapping implements memmap.Mappable.AddMapping.
-func (d *dentry) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) error {
+func (d *dentry) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) error {
d.mapsMu.Lock()
mapped := d.mappings.AddMapping(ms, ar, offset, writable)
// Do this unconditionally since whether we have a host FD can change
@@ -735,7 +738,7 @@ func (d *dentry) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar user
}
// RemoveMapping implements memmap.Mappable.RemoveMapping.
-func (d *dentry) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) {
+func (d *dentry) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) {
d.mapsMu.Lock()
unmapped := d.mappings.RemoveMapping(ms, ar, offset, writable)
for _, r := range unmapped {
@@ -759,12 +762,12 @@ func (d *dentry) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar u
}
// CopyMapping implements memmap.Mappable.CopyMapping.
-func (d *dentry) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64, writable bool) error {
+func (d *dentry) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR hostarch.AddrRange, offset uint64, writable bool) error {
return d.AddMapping(ctx, ms, dstAR, offset, writable)
}
// Translate implements memmap.Mappable.Translate.
-func (d *dentry) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
+func (d *dentry) Translate(ctx context.Context, required, optional memmap.MappableRange, at hostarch.AccessType) ([]memmap.Translation, error) {
d.handleMu.RLock()
if d.mmapFD >= 0 && !d.fs.opts.forcePageCache {
d.handleMu.RUnlock()
@@ -777,7 +780,7 @@ func (d *dentry) Translate(ctx context.Context, required, optional memmap.Mappab
Source: mr,
File: &d.pf,
Offset: mr.Start,
- Perms: usermem.AnyAccess,
+ Perms: hostarch.AnyAccess,
},
}, nil
}
@@ -786,7 +789,7 @@ func (d *dentry) Translate(ctx context.Context, required, optional memmap.Mappab
// Constrain translations to d.size (rounded up) to prevent translation to
// pages that may be concurrently truncated.
- pgend, _ := usermem.PageRoundUp(d.size)
+ pgend, _ := hostarch.PageRoundUp(d.size)
var beyondEOF bool
if required.End > pgend {
if required.Start >= pgend {
@@ -811,7 +814,7 @@ func (d *dentry) Translate(ctx context.Context, required, optional memmap.Mappab
segMR := seg.Range().Intersect(optional)
// TODO(jamieliu): Make Translations writable even if writability is
// not required if already kept-dirty by another writable translation.
- perms := usermem.AccessType{
+ perms := hostarch.AccessType{
Read: true,
Execute: true,
}
@@ -954,7 +957,7 @@ 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) {
+func (d *dentryPlatformFile) MapInternal(fr memmap.FileRange, at hostarch.AccessType) (safemem.BlockSeq, error) {
d.handleMu.RLock()
defer d.handleMu.RUnlock()
return d.hostFileMapper.MapInternal(fr, int(d.mmapFD), at.Write)
diff --git a/pkg/sentry/fsimpl/gofer/save_restore.go b/pkg/sentry/fsimpl/gofer/save_restore.go
index c90071e4e..83e841a51 100644
--- a/pkg/sentry/fsimpl/gofer/save_restore.go
+++ b/pkg/sentry/fsimpl/gofer/save_restore.go
@@ -22,12 +22,12 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fdnotifier"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
type saveRestoreContextID int
@@ -85,7 +85,7 @@ func (fs *filesystem) PrepareSave(ctx context.Context) error {
func (fd *specialFileFD) savePipeData(ctx context.Context) error {
fd.bufMu.Lock()
defer fd.bufMu.Unlock()
- var buf [usermem.PageSize]byte
+ var buf [hostarch.PageSize]byte
for {
n, err := fd.handle.readToBlocksAt(ctx, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf[:])), ^uint64(0))
if n != 0 {
diff --git a/pkg/sentry/fsimpl/host/BUILD b/pkg/sentry/fsimpl/host/BUILD
index 4ae9d6d5e..b94dfeb7f 100644
--- a/pkg/sentry/fsimpl/host/BUILD
+++ b/pkg/sentry/fsimpl/host/BUILD
@@ -47,6 +47,7 @@ go_library(
"//pkg/context",
"//pkg/fdnotifier",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/iovec",
"//pkg/log",
"//pkg/marshal/primitive",
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index b9cce4181..a81f550b1 100644
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -26,6 +26,7 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/hostfd"
@@ -431,8 +432,8 @@ func (i *inode) SetStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Cre
}
oldSize := uint64(hostStat.Size)
if s.Size < oldSize {
- oldpgend, _ := usermem.PageRoundUp(oldSize)
- newpgend, _ := usermem.PageRoundUp(s.Size)
+ oldpgend, _ := hostarch.PageRoundUp(oldSize)
+ newpgend, _ := hostarch.PageRoundUp(s.Size)
if oldpgend != newpgend {
i.CachedMappable.InvalidateRange(memmap.MappableRange{newpgend, oldpgend})
}
@@ -459,6 +460,9 @@ func (i *inode) DecRef(ctx context.Context) {
if err := unix.Close(i.hostFD); err != nil {
log.Warningf("failed to close host fd %d: %v", i.hostFD, err)
}
+ // We can't rely on fdnotifier when closing the fd, because the event may race
+ // with fdnotifier.RemoveFD. Instead, notify the queue explicitly.
+ i.queue.Notify(waiter.EventHUp | waiter.ReadableEvents | waiter.WritableEvents)
})
}
diff --git a/pkg/sentry/fsimpl/host/save_restore.go b/pkg/sentry/fsimpl/host/save_restore.go
index 5688bddc8..c502d8e99 100644
--- a/pkg/sentry/fsimpl/host/save_restore.go
+++ b/pkg/sentry/fsimpl/host/save_restore.go
@@ -21,9 +21,9 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/fdnotifier"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/hostfd"
- "gvisor.dev/gvisor/pkg/usermem"
)
// beforeSave is invoked by stateify.
@@ -38,7 +38,7 @@ func (i *inode) beforeSave() {
// EBADF from the read.
i.bufMu.Lock()
defer i.bufMu.Unlock()
- var buf [usermem.PageSize]byte
+ var buf [hostarch.PageSize]byte
for {
n, err := hostfd.Preadv2(int32(i.hostFD), safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf[:])), -1 /* offset */, 0 /* flags */)
if n != 0 {
@@ -68,3 +68,10 @@ func (i *inode) afterLoad() {
}
}
}
+
+// afterLoad is invoked by stateify.
+func (c *ConnectedEndpoint) afterLoad() {
+ if err := c.initFromOptions(); err != nil {
+ panic(fmt.Sprintf("initFromOptions failed: %v", err))
+ }
+}
diff --git a/pkg/sentry/fsimpl/host/socket.go b/pkg/sentry/fsimpl/host/socket.go
index 60e237ac7..ca85f5601 100644
--- a/pkg/sentry/fsimpl/host/socket.go
+++ b/pkg/sentry/fsimpl/host/socket.go
@@ -39,7 +39,7 @@ import (
func newEndpoint(ctx context.Context, hostFD int, queue *waiter.Queue) (transport.Endpoint, error) {
// Set up an external transport.Endpoint using the host fd.
addr := fmt.Sprintf("hostfd:[%d]", hostFD)
- e, err := NewConnectedEndpoint(ctx, hostFD, addr, true /* saveable */)
+ e, err := NewConnectedEndpoint(hostFD, addr)
if err != nil {
return nil, err.ToError()
}
@@ -86,7 +86,10 @@ type ConnectedEndpoint struct {
// for restoring them.
func (c *ConnectedEndpoint) init() *syserr.Error {
c.InitRefs()
+ return c.initFromOptions()
+}
+func (c *ConnectedEndpoint) initFromOptions() *syserr.Error {
family, err := unix.GetsockoptInt(c.fd, unix.SOL_SOCKET, unix.SO_DOMAIN)
if err != nil {
return syserr.FromError(err)
@@ -123,7 +126,7 @@ func (c *ConnectedEndpoint) init() *syserr.Error {
// The caller is responsible for calling Init(). Additionaly, Release needs to
// be called twice because ConnectedEndpoint is both a transport.Receiver and
// transport.ConnectedEndpoint.
-func NewConnectedEndpoint(ctx context.Context, hostFD int, addr string, saveable bool) (*ConnectedEndpoint, *syserr.Error) {
+func NewConnectedEndpoint(hostFD int, addr string) (*ConnectedEndpoint, *syserr.Error) {
e := ConnectedEndpoint{
fd: hostFD,
addr: addr,
@@ -330,8 +333,16 @@ func (c *ConnectedEndpoint) CloseUnread() {}
// SetSendBufferSize implements transport.ConnectedEndpoint.SetSendBufferSize.
func (c *ConnectedEndpoint) SetSendBufferSize(v int64) (newSz int64) {
- // gVisor does not permit setting of SO_SNDBUF for host backed unix domain
- // sockets.
+ // gVisor does not permit setting of SO_SNDBUF for host backed unix
+ // domain sockets.
+ return atomic.LoadInt64(&c.sndbuf)
+}
+
+// SetReceiveBufferSize implements transport.ConnectedEndpoint.SetReceiveBufferSize.
+func (c *ConnectedEndpoint) SetReceiveBufferSize(v int64) (newSz int64) {
+ // gVisor does not permit setting of SO_RCVBUF for host backed unix
+ // domain sockets. Receive buffer does not have any effect for unix
+ // sockets and we claim to be the same as send buffer.
return atomic.LoadInt64(&c.sndbuf)
}
diff --git a/pkg/sentry/fsimpl/kernfs/BUILD b/pkg/sentry/fsimpl/kernfs/BUILD
index 6dbc7e34d..b7d13cced 100644
--- a/pkg/sentry/fsimpl/kernfs/BUILD
+++ b/pkg/sentry/fsimpl/kernfs/BUILD
@@ -105,6 +105,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
diff --git a/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go b/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
index 65054b0ea..84b1c3745 100644
--- a/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
+++ b/pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go
@@ -25,8 +25,10 @@ import (
"gvisor.dev/gvisor/pkg/usermem"
)
-// DynamicBytesFile implements kernfs.Inode and represents a read-only
-// file whose contents are backed by a vfs.DynamicBytesSource.
+// DynamicBytesFile implements kernfs.Inode and represents a read-only file
+// whose contents are backed by a vfs.DynamicBytesSource. If data additionally
+// implements vfs.WritableDynamicBytesSource, the file also supports dispatching
+// writes to the implementer, but note that this will not update the source data.
//
// Must be instantiated with NewDynamicBytesFile or initialized with Init
// before first use.
@@ -40,7 +42,9 @@ type DynamicBytesFile struct {
InodeNotSymlink
locks vfs.FileLocks
- data vfs.DynamicBytesSource
+ // data can additionally implement vfs.WritableDynamicBytesSource to support
+ // writes.
+ data vfs.DynamicBytesSource
}
var _ Inode = (*DynamicBytesFile)(nil)
diff --git a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
index 6b890a39c..3d0866ecf 100644
--- a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
+++ b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go
@@ -20,12 +20,12 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
// InodeNoopRefCount partially implements the Inode interface, specifically the
@@ -206,7 +206,7 @@ func (a *InodeAttrs) Init(ctx context.Context, creds *auth.Credentials, devMajor
atomic.StoreUint32(&a.uid, uint32(creds.EffectiveKUID))
atomic.StoreUint32(&a.gid, uint32(creds.EffectiveKGID))
atomic.StoreUint32(&a.nlink, nlink)
- atomic.StoreUint32(&a.blockSize, usermem.PageSize)
+ atomic.StoreUint32(&a.blockSize, hostarch.PageSize)
now := ktime.NowFromContext(ctx).Nanoseconds()
atomic.StoreInt64(&a.atime, now)
atomic.StoreInt64(&a.mtime, now)
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go
index 565d723f0..16486eeae 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs.go
@@ -61,6 +61,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
@@ -508,6 +509,15 @@ func (d *Dentry) Inode() Inode {
return d.inode
}
+// FSLocalPath returns an absolute path to d, relative to the root of its
+// filesystem.
+func (d *Dentry) FSLocalPath() string {
+ var b fspath.Builder
+ _ = genericPrependPath(vfs.VirtualDentry{}, nil, d, &b)
+ b.PrependByte('/')
+ return b.String()
+}
+
// The Inode interface maps filesystem-level operations that operate on paths to
// equivalent operations on specific filesystem nodes.
//
diff --git a/pkg/sentry/fsimpl/kernfs/mmap_util.go b/pkg/sentry/fsimpl/kernfs/mmap_util.go
index bd6a134b4..d1539d904 100644
--- a/pkg/sentry/fsimpl/kernfs/mmap_util.go
+++ b/pkg/sentry/fsimpl/kernfs/mmap_util.go
@@ -16,11 +16,11 @@ package kernfs
import (
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sync"
- "gvisor.dev/gvisor/pkg/usermem"
)
// inodePlatformFile implements memmap.File. It exists solely because inode
@@ -66,7 +66,7 @@ func (i *inodePlatformFile) DecRef(fr memmap.FileRange) {
}
// MapInternal implements memmap.File.MapInternal.
-func (i *inodePlatformFile) MapInternal(fr memmap.FileRange, at usermem.AccessType) (safemem.BlockSeq, error) {
+func (i *inodePlatformFile) MapInternal(fr memmap.FileRange, at hostarch.AccessType) (safemem.BlockSeq, error) {
return i.fileMapper.MapInternal(fr, i.hostFD, at.Write)
}
@@ -100,7 +100,7 @@ func (i *CachedMappable) Init(hostFD int) {
}
// AddMapping implements memmap.Mappable.AddMapping.
-func (i *CachedMappable) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) error {
+func (i *CachedMappable) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) error {
i.mapsMu.Lock()
mapped := i.mappings.AddMapping(ms, ar, offset, writable)
for _, r := range mapped {
@@ -111,7 +111,7 @@ func (i *CachedMappable) AddMapping(ctx context.Context, ms memmap.MappingSpace,
}
// RemoveMapping implements memmap.Mappable.RemoveMapping.
-func (i *CachedMappable) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) {
+func (i *CachedMappable) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) {
i.mapsMu.Lock()
unmapped := i.mappings.RemoveMapping(ms, ar, offset, writable)
for _, r := range unmapped {
@@ -121,19 +121,19 @@ func (i *CachedMappable) RemoveMapping(ctx context.Context, ms memmap.MappingSpa
}
// CopyMapping implements memmap.Mappable.CopyMapping.
-func (i *CachedMappable) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64, writable bool) error {
+func (i *CachedMappable) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR hostarch.AddrRange, offset uint64, writable bool) error {
return i.AddMapping(ctx, ms, dstAR, offset, writable)
}
// Translate implements memmap.Mappable.Translate.
-func (i *CachedMappable) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
+func (i *CachedMappable) Translate(ctx context.Context, required, optional memmap.MappableRange, at hostarch.AccessType) ([]memmap.Translation, error) {
mr := optional
return []memmap.Translation{
{
Source: mr,
File: &i.pf,
Offset: mr.Start,
- Perms: usermem.AnyAccess,
+ Perms: hostarch.AnyAccess,
},
}, nil
}
diff --git a/pkg/sentry/fsimpl/overlay/BUILD b/pkg/sentry/fsimpl/overlay/BUILD
index bf13bbbf4..5504476c8 100644
--- a/pkg/sentry/fsimpl/overlay/BUILD
+++ b/pkg/sentry/fsimpl/overlay/BUILD
@@ -30,6 +30,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
diff --git a/pkg/sentry/fsimpl/overlay/copy_up.go b/pkg/sentry/fsimpl/overlay/copy_up.go
index 27b00cf6f..45aa5a494 100644
--- a/pkg/sentry/fsimpl/overlay/copy_up.go
+++ b/pkg/sentry/fsimpl/overlay/copy_up.go
@@ -21,11 +21,11 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
func (d *dentry) isCopiedUp() bool {
@@ -138,8 +138,8 @@ func (d *dentry) copyUpLocked(ctx context.Context) error {
// We may have memory mappings of the file on the lower layer.
// Switch to mapping the file on the upper layer instead.
mmapOpts = &memmap.MMapOpts{
- Perms: usermem.ReadWrite,
- MaxPerms: usermem.ReadWrite,
+ Perms: hostarch.ReadWrite,
+ MaxPerms: hostarch.ReadWrite,
}
if err := newFD.ConfigureMMap(ctx, mmapOpts); err != nil {
cleanupUndoCopyUp()
diff --git a/pkg/sentry/fsimpl/overlay/regular_file.go b/pkg/sentry/fsimpl/overlay/regular_file.go
index d791c06db..43bfd69a3 100644
--- a/pkg/sentry/fsimpl/overlay/regular_file.go
+++ b/pkg/sentry/fsimpl/overlay/regular_file.go
@@ -19,6 +19,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -445,7 +446,7 @@ func (fd *regularFileFD) ensureMappable(ctx context.Context, opts *memmap.MMapOp
}
// AddMapping implements memmap.Mappable.AddMapping.
-func (d *dentry) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) error {
+func (d *dentry) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) error {
d.mapsMu.Lock()
defer d.mapsMu.Unlock()
if err := d.wrappedMappable.AddMapping(ctx, ms, ar, offset, writable); err != nil {
@@ -458,7 +459,7 @@ func (d *dentry) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar user
}
// RemoveMapping implements memmap.Mappable.RemoveMapping.
-func (d *dentry) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) {
+func (d *dentry) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) {
d.mapsMu.Lock()
defer d.mapsMu.Unlock()
d.wrappedMappable.RemoveMapping(ctx, ms, ar, offset, writable)
@@ -468,7 +469,7 @@ func (d *dentry) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar u
}
// CopyMapping implements memmap.Mappable.CopyMapping.
-func (d *dentry) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64, writable bool) error {
+func (d *dentry) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR hostarch.AddrRange, offset uint64, writable bool) error {
d.mapsMu.Lock()
defer d.mapsMu.Unlock()
if err := d.wrappedMappable.CopyMapping(ctx, ms, srcAR, dstAR, offset, writable); err != nil {
@@ -481,7 +482,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) {
+func (d *dentry) Translate(ctx context.Context, required, optional memmap.MappableRange, at hostarch.AccessType) ([]memmap.Translation, error) {
d.dataMu.RLock()
defer d.dataMu.RUnlock()
return d.wrappedMappable.Translate(ctx, required, optional, at)
diff --git a/pkg/sentry/fsimpl/pipefs/BUILD b/pkg/sentry/fsimpl/pipefs/BUILD
index 5950a2d59..278ee3c92 100644
--- a/pkg/sentry/fsimpl/pipefs/BUILD
+++ b/pkg/sentry/fsimpl/pipefs/BUILD
@@ -10,12 +10,12 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/sentry/fsimpl/kernfs",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/time",
"//pkg/sentry/vfs",
"//pkg/syserror",
- "//pkg/usermem",
],
)
diff --git a/pkg/sentry/fsimpl/pipefs/pipefs.go b/pkg/sentry/fsimpl/pipefs/pipefs.go
index 3f05e444e..08aedc2ad 100644
--- a/pkg/sentry/fsimpl/pipefs/pipefs.go
+++ b/pkg/sentry/fsimpl/pipefs/pipefs.go
@@ -22,13 +22,13 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
// +stateify savable
@@ -131,7 +131,7 @@ func (i *inode) Stat(_ context.Context, vfsfs *vfs.Filesystem, opts vfs.StatOpti
ts := linux.NsecToStatxTimestamp(i.ctime.Nanoseconds())
return linux.Statx{
Mask: linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_NLINK | linux.STATX_UID | linux.STATX_GID | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME | linux.STATX_INO | linux.STATX_SIZE | linux.STATX_BLOCKS,
- Blksize: usermem.PageSize,
+ Blksize: hostarch.PageSize,
Nlink: 1,
UID: uint32(i.uid),
GID: uint32(i.gid),
diff --git a/pkg/sentry/fsimpl/proc/BUILD b/pkg/sentry/fsimpl/proc/BUILD
index d47a4fff9..2b628bd55 100644
--- a/pkg/sentry/fsimpl/proc/BUILD
+++ b/pkg/sentry/fsimpl/proc/BUILD
@@ -81,6 +81,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
diff --git a/pkg/sentry/fsimpl/proc/filesystem.go b/pkg/sentry/fsimpl/proc/filesystem.go
index 254a8b062..ce8f55b1f 100644
--- a/pkg/sentry/fsimpl/proc/filesystem.go
+++ b/pkg/sentry/fsimpl/proc/filesystem.go
@@ -86,13 +86,13 @@ func (ft FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualF
procfs.MaxCachedDentries = maxCachedDentries
procfs.VFSFilesystem().Init(vfsObj, &ft, procfs)
- var cgroups map[string]string
+ var fakeCgroupControllers map[string]string
if opts.InternalData != nil {
data := opts.InternalData.(*InternalData)
- cgroups = data.Cgroups
+ fakeCgroupControllers = data.Cgroups
}
- inode := procfs.newTasksInode(ctx, k, pidns, cgroups)
+ inode := procfs.newTasksInode(ctx, k, pidns, fakeCgroupControllers)
var dentry kernfs.Dentry
dentry.InitRoot(&procfs.Filesystem, inode)
return procfs.VFSFilesystem(), dentry.VFSDentry(), nil
diff --git a/pkg/sentry/fsimpl/proc/task.go b/pkg/sentry/fsimpl/proc/task.go
index fea138f93..d05cc1508 100644
--- a/pkg/sentry/fsimpl/proc/task.go
+++ b/pkg/sentry/fsimpl/proc/task.go
@@ -47,7 +47,7 @@ type taskInode struct {
var _ kernfs.Inode = (*taskInode)(nil)
-func (fs *filesystem) newTaskInode(ctx context.Context, task *kernel.Task, pidns *kernel.PIDNamespace, isThreadGroup bool, cgroupControllers map[string]string) (kernfs.Inode, error) {
+func (fs *filesystem) newTaskInode(ctx context.Context, task *kernel.Task, pidns *kernel.PIDNamespace, isThreadGroup bool, fakeCgroupControllers map[string]string) (kernfs.Inode, error) {
if task.ExitState() == kernel.TaskExitDead {
return nil, syserror.ESRCH
}
@@ -82,10 +82,12 @@ func (fs *filesystem) newTaskInode(ctx context.Context, task *kernel.Task, pidns
"uid_map": fs.newTaskOwnedInode(ctx, task, fs.NextIno(), 0644, &idMapData{task: task, gids: false}),
}
if isThreadGroup {
- contents["task"] = fs.newSubtasks(ctx, task, pidns, cgroupControllers)
+ contents["task"] = fs.newSubtasks(ctx, task, pidns, fakeCgroupControllers)
}
- if len(cgroupControllers) > 0 {
- contents["cgroup"] = fs.newTaskOwnedInode(ctx, task, fs.NextIno(), 0444, newCgroupData(cgroupControllers))
+ if len(fakeCgroupControllers) > 0 {
+ contents["cgroup"] = fs.newTaskOwnedInode(ctx, task, fs.NextIno(), 0444, newFakeCgroupData(fakeCgroupControllers))
+ } else {
+ contents["cgroup"] = fs.newTaskOwnedInode(ctx, task, fs.NextIno(), 0444, &taskCgroupData{task: task})
}
taskInode := &taskInode{task: task}
@@ -226,11 +228,14 @@ func newIO(t *kernel.Task, isThreadGroup bool) *ioData {
return &ioData{ioUsage: t}
}
-// newCgroupData creates inode that shows cgroup information.
-// From man 7 cgroups: "For each cgroup hierarchy of which the process is a
-// member, there is one entry containing three colon-separated fields:
-// hierarchy-ID:controller-list:cgroup-path"
-func newCgroupData(controllers map[string]string) dynamicInode {
+// newFakeCgroupData creates an inode that shows fake cgroup
+// information passed in as mount options. From man 7 cgroups: "For
+// each cgroup hierarchy of which the process is a member, there is
+// one entry containing three colon-separated fields:
+// hierarchy-ID:controller-list:cgroup-path"
+//
+// TODO(b/182488796): Remove once all users adopt cgroupfs.
+func newFakeCgroupData(controllers map[string]string) dynamicInode {
var buf bytes.Buffer
// The hierarchy ids must be positive integers (for cgroup v1), but the
diff --git a/pkg/sentry/fsimpl/proc/task_files.go b/pkg/sentry/fsimpl/proc/task_files.go
index fdae163d1..b294dfd6a 100644
--- a/pkg/sentry/fsimpl/proc/task_files.go
+++ b/pkg/sentry/fsimpl/proc/task_files.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/fsbridge"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
@@ -122,8 +123,8 @@ func (d *auxvData) Generate(ctx context.Context, buf *bytes.Buffer) error {
buf.Grow((len(auxv) + 1) * 16)
for _, e := range auxv {
var tmp [16]byte
- usermem.ByteOrder.PutUint64(tmp[:8], e.Key)
- usermem.ByteOrder.PutUint64(tmp[8:], uint64(e.Value))
+ hostarch.ByteOrder.PutUint64(tmp[:8], e.Key)
+ hostarch.ByteOrder.PutUint64(tmp[8:], uint64(e.Value))
buf.Write(tmp[:])
}
var atNull [16]byte
@@ -168,15 +169,15 @@ func (d *cmdlineData) Generate(ctx context.Context, buf *bytes.Buffer) error {
defer m.DecUsers(ctx)
// Figure out the bounds of the exec arg we are trying to read.
- var ar usermem.AddrRange
+ var ar hostarch.AddrRange
switch d.arg {
case cmdlineDataArg:
- ar = usermem.AddrRange{
+ ar = hostarch.AddrRange{
Start: m.ArgvStart(),
End: m.ArgvEnd(),
}
case environDataArg:
- ar = usermem.AddrRange{
+ ar = hostarch.AddrRange{
Start: m.EnvvStart(),
End: m.EnvvEnd(),
}
@@ -192,7 +193,7 @@ func (d *cmdlineData) Generate(ctx context.Context, buf *bytes.Buffer) error {
// until Linux 4.9 (272ddc8b3735 "proc: don't use FOLL_FORCE for reading
// cmdline and environment").
writer := &bufferWriter{buf: buf}
- if n, err := m.CopyInTo(ctx, usermem.AddrRangeSeqOf(ar), writer, usermem.IOOpts{}); n == 0 || err != nil {
+ if n, err := m.CopyInTo(ctx, hostarch.AddrRangeSeqOf(ar), writer, usermem.IOOpts{}); n == 0 || err != nil {
// Nothing to copy or something went wrong.
return err
}
@@ -209,7 +210,7 @@ func (d *cmdlineData) Generate(ctx context.Context, buf *bytes.Buffer) error {
}
// There is no NULL terminator in the string, return into envp.
- arEnvv := usermem.AddrRange{
+ arEnvv := hostarch.AddrRange{
Start: m.EnvvStart(),
End: m.EnvvEnd(),
}
@@ -218,11 +219,11 @@ func (d *cmdlineData) Generate(ctx context.Context, buf *bytes.Buffer) error {
// https://elixir.bootlin.com/linux/v4.20/source/fs/proc/base.c#L208
// we'll return one page total between argv and envp because of the
// above page restrictions.
- if buf.Len() >= usermem.PageSize {
+ if buf.Len() >= hostarch.PageSize {
// Returned at least one page already, nothing else to add.
return nil
}
- remaining := usermem.PageSize - buf.Len()
+ remaining := hostarch.PageSize - buf.Len()
if int(arEnvv.Length()) > remaining {
end, ok := arEnvv.Start.AddLength(uint64(remaining))
if !ok {
@@ -230,7 +231,7 @@ func (d *cmdlineData) Generate(ctx context.Context, buf *bytes.Buffer) error {
}
arEnvv.End = end
}
- if _, err := m.CopyInTo(ctx, usermem.AddrRangeSeqOf(arEnvv), writer, usermem.IOOpts{}); err != nil {
+ if _, err := m.CopyInTo(ctx, hostarch.AddrRangeSeqOf(arEnvv), writer, usermem.IOOpts{}); err != nil {
return err
}
@@ -323,7 +324,7 @@ func (d *idMapData) Write(ctx context.Context, src usermem.IOSequence, offset in
// the system page size, and the write must be performed at the start of
// the file ..." - user_namespaces(7)
srclen := src.NumBytes()
- if srclen >= usermem.PageSize || offset != 0 {
+ if srclen >= hostarch.PageSize || offset != 0 {
return 0, syserror.EINVAL
}
b := make([]byte, srclen)
@@ -481,7 +482,7 @@ func (fd *memFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64
defer m.DecUsers(ctx)
// Buffer the read data because of MM locks
buf := make([]byte, dst.NumBytes())
- n, readErr := m.CopyIn(ctx, usermem.Addr(offset), buf, usermem.IOOpts{IgnorePermissions: true})
+ n, readErr := m.CopyIn(ctx, hostarch.Addr(offset), buf, usermem.IOOpts{IgnorePermissions: true})
if n > 0 {
if _, err := dst.CopyOut(ctx, buf[:n]); err != nil {
return 0, syserror.EFAULT
@@ -613,7 +614,7 @@ func (s *taskStatData) Generate(ctx context.Context, buf *bytes.Buffer) error {
rss = mm.ResidentSetSize()
}
})
- fmt.Fprintf(buf, "%d %d ", vss, rss/usermem.PageSize)
+ fmt.Fprintf(buf, "%d %d ", vss, rss/hostarch.PageSize)
// rsslim.
fmt.Fprintf(buf, "%d ", s.task.ThreadGroup().Limits().Get(limits.Rss).Cur)
@@ -655,7 +656,7 @@ func (s *statmData) Generate(ctx context.Context, buf *bytes.Buffer) error {
}
})
- fmt.Fprintf(buf, "%d %d 0 0 0 0 0\n", vss/usermem.PageSize, rss/usermem.PageSize)
+ fmt.Fprintf(buf, "%d %d 0 0 0 0 0\n", vss/hostarch.PageSize, rss/hostarch.PageSize)
return nil
}
@@ -774,7 +775,7 @@ func (o *oomScoreAdj) Write(ctx context.Context, src usermem.IOSequence, offset
}
// Limit input size so as not to impact performance if input size is large.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
var v int32
n, err := usermem.CopyInt32StringInVec(ctx, src.IO, src.Addrs, &v, src.Opts)
@@ -1099,3 +1100,32 @@ func (fd *namespaceFD) SetStat(ctx context.Context, opts vfs.SetStatOptions) err
func (fd *namespaceFD) Release(ctx context.Context) {
fd.inode.DecRef(ctx)
}
+
+// taskCgroupData generates data for /proc/[pid]/cgroup.
+//
+// +stateify savable
+type taskCgroupData struct {
+ dynamicBytesFileSetAttr
+ task *kernel.Task
+}
+
+var _ dynamicInode = (*taskCgroupData)(nil)
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *taskCgroupData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ // When a task is existing on Linux, a task's cgroup set is cleared and
+ // reset to the initial cgroup set, which is essentially the set of root
+ // cgroups. Because of this, the /proc/<pid>/cgroup file is always readable
+ // on Linux throughout a task's lifetime.
+ //
+ // The sentry removes tasks from cgroups during the exit process, but
+ // doesn't move them into an initial cgroup set, so partway through task
+ // exit this file show a task is in no cgroups, which is incorrect. Instead,
+ // once a task has left its cgroups, we return an error.
+ if d.task.ExitState() >= kernel.TaskExitInitiated {
+ return syserror.ESRCH
+ }
+
+ d.task.GenerateProcTaskCgroup(buf)
+ return nil
+}
diff --git a/pkg/sentry/fsimpl/proc/task_net.go b/pkg/sentry/fsimpl/proc/task_net.go
index d4f6a5a9b..177cb828f 100644
--- a/pkg/sentry/fsimpl/proc/task_net.go
+++ b/pkg/sentry/fsimpl/proc/task_net.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/inet"
@@ -34,7 +35,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/tcpip/header"
- "gvisor.dev/gvisor/pkg/usermem"
)
func (fs *filesystem) newTaskNetDir(ctx context.Context, task *kernel.Task) kernfs.Inode {
@@ -295,7 +295,7 @@ func networkToHost16(n uint16) uint16 {
// binary.BigEndian.Uint16() require a read of binary.BigEndian and an
// interface method call, defeating inlining.
buf := [2]byte{byte(n >> 8 & 0xff), byte(n & 0xff)}
- return usermem.ByteOrder.Uint16(buf[:])
+ return hostarch.ByteOrder.Uint16(buf[:])
}
func writeInetAddr(w io.Writer, family int, i linux.SockAddr) {
@@ -317,14 +317,14 @@ func writeInetAddr(w io.Writer, family int, i linux.SockAddr) {
// __be32 which is a typedef for an unsigned int, and is printed with
// %X. This means that for a little-endian machine, Linux prints the
// least-significant byte of the address first. To emulate this, we first
- // invert the byte order for the address using usermem.ByteOrder.Uint32,
+ // invert the byte order for the address using hostarch.ByteOrder.Uint32,
// which makes it have the equivalent encoding to a __be32 on a little
// endian machine. Note that this operation is a no-op on a big endian
// machine. Then similar to Linux, we format it with %X, which will print
// the most-significant byte of the __be32 address first, which is now
// actually the least-significant byte of the original address in
// linux.SockAddrInet.Addr on little endian machines, due to the conversion.
- addr := usermem.ByteOrder.Uint32(a.Addr[:])
+ addr := hostarch.ByteOrder.Uint32(a.Addr[:])
fmt.Fprintf(w, "%08X:%04X ", addr, port)
case linux.AF_INET6:
@@ -334,10 +334,10 @@ func writeInetAddr(w io.Writer, family int, i linux.SockAddr) {
}
port := networkToHost16(a.Port)
- addr0 := usermem.ByteOrder.Uint32(a.Addr[0:4])
- addr1 := usermem.ByteOrder.Uint32(a.Addr[4:8])
- addr2 := usermem.ByteOrder.Uint32(a.Addr[8:12])
- addr3 := usermem.ByteOrder.Uint32(a.Addr[12:16])
+ addr0 := hostarch.ByteOrder.Uint32(a.Addr[0:4])
+ addr1 := hostarch.ByteOrder.Uint32(a.Addr[4:8])
+ addr2 := hostarch.ByteOrder.Uint32(a.Addr[8:12])
+ addr3 := hostarch.ByteOrder.Uint32(a.Addr[12:16])
fmt.Fprintf(w, "%08X%08X%08X%08X:%04X ", addr0, addr1, addr2, addr3, port)
}
}
@@ -739,10 +739,10 @@ func (d *netRouteData) Generate(ctx context.Context, buf *bytes.Buffer) error {
)
if len(rt.GatewayAddr) == header.IPv4AddressSize {
flags |= linux.RTF_GATEWAY
- gw = usermem.ByteOrder.Uint32(rt.GatewayAddr)
+ gw = hostarch.ByteOrder.Uint32(rt.GatewayAddr)
}
if len(rt.DstAddr) == header.IPv4AddressSize {
- prefix = usermem.ByteOrder.Uint32(rt.DstAddr)
+ prefix = hostarch.ByteOrder.Uint32(rt.DstAddr)
}
l := fmt.Sprintf(
"%s\t%08X\t%08X\t%04X\t%d\t%d\t%d\t%08X\t%d\t%d\t%d",
diff --git a/pkg/sentry/fsimpl/proc/tasks.go b/pkg/sentry/fsimpl/proc/tasks.go
index fdc580610..7c7543f14 100644
--- a/pkg/sentry/fsimpl/proc/tasks.go
+++ b/pkg/sentry/fsimpl/proc/tasks.go
@@ -54,15 +54,15 @@ type tasksInode struct {
// '/proc/self' and '/proc/thread-self' have custom directory offsets in
// Linux. So handle them outside of OrderedChildren.
- // cgroupControllers is a map of controller name to directory in the
+ // fakeCgroupControllers is a map of controller name to directory in the
// cgroup hierarchy. These controllers are immutable and will be listed
// in /proc/pid/cgroup if not nil.
- cgroupControllers map[string]string
+ fakeCgroupControllers map[string]string
}
var _ kernfs.Inode = (*tasksInode)(nil)
-func (fs *filesystem) newTasksInode(ctx context.Context, k *kernel.Kernel, pidns *kernel.PIDNamespace, cgroupControllers map[string]string) *tasksInode {
+func (fs *filesystem) newTasksInode(ctx context.Context, k *kernel.Kernel, pidns *kernel.PIDNamespace, fakeCgroupControllers map[string]string) *tasksInode {
root := auth.NewRootCredentials(pidns.UserNamespace())
contents := map[string]kernfs.Inode{
"cpuinfo": fs.newInode(ctx, root, 0444, newStaticFileSetStat(cpuInfoData(k))),
@@ -76,11 +76,16 @@ func (fs *filesystem) newTasksInode(ctx context.Context, k *kernel.Kernel, pidns
"uptime": fs.newInode(ctx, root, 0444, &uptimeData{}),
"version": fs.newInode(ctx, root, 0444, &versionData{}),
}
+ // If fakeCgroupControllers are provided, don't create a cgroupfs backed
+ // /proc/cgroup as it will not match the fake controllers.
+ if len(fakeCgroupControllers) == 0 {
+ contents["cgroups"] = fs.newInode(ctx, root, 0444, &cgroupsData{})
+ }
inode := &tasksInode{
- pidns: pidns,
- fs: fs,
- cgroupControllers: cgroupControllers,
+ pidns: pidns,
+ fs: fs,
+ fakeCgroupControllers: fakeCgroupControllers,
}
inode.InodeAttrs.Init(ctx, root, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0555)
inode.InitRefs()
@@ -118,7 +123,7 @@ func (i *tasksInode) Lookup(ctx context.Context, name string) (kernfs.Inode, err
return nil, syserror.ENOENT
}
- return i.fs.newTaskInode(ctx, task, i.pidns, true, i.cgroupControllers)
+ return i.fs.newTaskInode(ctx, task, i.pidns, true, i.fakeCgroupControllers)
}
// IterDirents implements kernfs.inodeDirectory.IterDirents.
diff --git a/pkg/sentry/fsimpl/proc/tasks_files.go b/pkg/sentry/fsimpl/proc/tasks_files.go
index 01b7a6678..e1a8b4409 100644
--- a/pkg/sentry/fsimpl/proc/tasks_files.go
+++ b/pkg/sentry/fsimpl/proc/tasks_files.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -28,7 +29,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/usage"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
// +stateify savable
@@ -270,7 +270,7 @@ func (*meminfoData) Generate(ctx context.Context, buf *bytes.Buffer) error {
anon := snapshot.Anonymous + snapshot.Tmpfs
file := snapshot.PageCache + snapshot.Mapped
// We don't actually have active/inactive LRUs, so just make up numbers.
- activeFile := (file / 2) &^ (usermem.PageSize - 1)
+ activeFile := (file / 2) &^ (hostarch.PageSize - 1)
inactiveFile := file - activeFile
fmt.Fprintf(buf, "MemTotal: %8d kB\n", totalSize/1024)
@@ -384,3 +384,19 @@ func (d *filesystemsData) Generate(ctx context.Context, buf *bytes.Buffer) error
k.VFS().GenerateProcFilesystems(buf)
return nil
}
+
+// cgroupsData backs /proc/cgroups.
+//
+// +stateify savable
+type cgroupsData struct {
+ dynamicBytesFileSetAttr
+}
+
+var _ dynamicInode = (*cgroupsData)(nil)
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (*cgroupsData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ r := kernel.KernelFromContext(ctx).CgroupRegistry()
+ r.GenerateProcCgroups(buf)
+ return nil
+}
diff --git a/pkg/sentry/fsimpl/proc/tasks_sys.go b/pkg/sentry/fsimpl/proc/tasks_sys.go
index fb274b78e..9b14dd6b9 100644
--- a/pkg/sentry/fsimpl/proc/tasks_sys.go
+++ b/pkg/sentry/fsimpl/proc/tasks_sys.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -214,7 +215,7 @@ func (d *tcpSackData) Write(ctx context.Context, src usermem.IOSequence, offset
}
// Limit the amount of memory allocated.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
var v int32
n, err := usermem.CopyInt32StringInVec(ctx, src.IO, src.Addrs, &v, src.Opts)
@@ -262,7 +263,7 @@ func (d *tcpRecoveryData) Write(ctx context.Context, src usermem.IOSequence, off
}
// Limit the amount of memory allocated.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
var v int32
n, err := usermem.CopyInt32StringInVec(ctx, src.IO, src.Addrs, &v, src.Opts)
@@ -318,7 +319,7 @@ func (d *tcpMemData) Write(ctx context.Context, src usermem.IOSequence, offset i
defer d.mu.Unlock()
// Limit the amount of memory allocated.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
size, err := d.readSizeLocked()
if err != nil {
return 0, err
@@ -406,7 +407,7 @@ func (ipf *ipForwarding) Write(ctx context.Context, src usermem.IOSequence, offs
}
// Limit input size so as not to impact performance if input size is large.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
var v int32
n, err := usermem.CopyInt32StringInVec(ctx, src.IO, src.Addrs, &v, src.Opts)
@@ -463,7 +464,7 @@ func (pr *portRange) Write(ctx context.Context, src usermem.IOSequence, offset i
// Limit input size so as not to impact performance if input size is
// large.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
ports := make([]int32, 2)
n, err := usermem.CopyInt32StringsInVec(ctx, src.IO, src.Addrs, ports, src.Opts)
diff --git a/pkg/sentry/fsimpl/proc/yama.go b/pkg/sentry/fsimpl/proc/yama.go
index aebfe8944..e039ec45e 100644
--- a/pkg/sentry/fsimpl/proc/yama.go
+++ b/pkg/sentry/fsimpl/proc/yama.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -62,7 +63,7 @@ func (s *yamaPtraceScope) Write(ctx context.Context, src usermem.IOSequence, off
}
// Limit the amount of memory allocated.
- src = src.TakeFirst(usermem.PageSize - 1)
+ src = src.TakeFirst(hostarch.PageSize - 1)
var v int32
n, err := usermem.CopyInt32StringInVec(ctx, src.IO, src.Addrs, &v, src.Opts)
diff --git a/pkg/sentry/fsimpl/sys/sys.go b/pkg/sentry/fsimpl/sys/sys.go
index 1d9280dae..14eb10dcd 100644
--- a/pkg/sentry/fsimpl/sys/sys.go
+++ b/pkg/sentry/fsimpl/sys/sys.go
@@ -122,11 +122,11 @@ func cpuDir(ctx context.Context, fs *filesystem, creds *auth.Credentials) kernfs
}
func kernelDir(ctx context.Context, fs *filesystem, creds *auth.Credentials) kernfs.Inode {
- // If kcov is available, set up /sys/kernel/debug/kcov. Technically, debugfs
- // should be mounted at debug/, but for our purposes, it is sufficient to
- // keep it in sys.
+ // Set up /sys/kernel/debug/kcov. Technically, debugfs should be
+ // mounted at debug/, but for our purposes, it is sufficient to keep it
+ // in sys.
var children map[string]kernfs.Inode
- if coverage.KcovAvailable() {
+ if coverage.KcovSupported() {
log.Debugf("Set up /sys/kernel/debug/kcov")
children = map[string]kernfs.Inode{
"debug": fs.newDir(ctx, creds, linux.FileMode(0700), map[string]kernfs.Inode{
diff --git a/pkg/sentry/fsimpl/testutil/BUILD b/pkg/sentry/fsimpl/testutil/BUILD
index 400a97996..b3f9d1010 100644
--- a/pkg/sentry/fsimpl/testutil/BUILD
+++ b/pkg/sentry/fsimpl/testutil/BUILD
@@ -15,6 +15,7 @@ go_library(
"//pkg/context",
"//pkg/cpuid",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/memutil",
"//pkg/sentry/fsbridge",
"//pkg/sentry/fsimpl/tmpfs",
diff --git a/pkg/sentry/fsimpl/testutil/testutil.go b/pkg/sentry/fsimpl/testutil/testutil.go
index 1a8525b06..59e6f9c92 100644
--- a/pkg/sentry/fsimpl/testutil/testutil.go
+++ b/pkg/sentry/fsimpl/testutil/testutil.go
@@ -30,6 +30,8 @@ import (
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/usermem"
+
+ "gvisor.dev/gvisor/pkg/hostarch"
)
// System represents the context for a single test.
@@ -105,7 +107,7 @@ func (s *System) Destroy() {
// ReadToEnd reads the contents of fd until EOF to a string.
func (s *System) ReadToEnd(fd *vfs.FileDescription) (string, error) {
- buf := make([]byte, usermem.PageSize)
+ buf := make([]byte, hostarch.PageSize)
bufIOSeq := usermem.BytesIOSequence(buf)
opts := vfs.ReadOptions{}
diff --git a/pkg/sentry/fsimpl/timerfd/BUILD b/pkg/sentry/fsimpl/timerfd/BUILD
index fbb02a271..7ce7dc429 100644
--- a/pkg/sentry/fsimpl/timerfd/BUILD
+++ b/pkg/sentry/fsimpl/timerfd/BUILD
@@ -8,6 +8,7 @@ go_library(
visibility = ["//pkg/sentry:internal"],
deps = [
"//pkg/context",
+ "//pkg/hostarch",
"//pkg/sentry/kernel/time",
"//pkg/sentry/vfs",
"//pkg/syserror",
diff --git a/pkg/sentry/fsimpl/timerfd/timerfd.go b/pkg/sentry/fsimpl/timerfd/timerfd.go
index 64d33c3a8..cbb8b67c5 100644
--- a/pkg/sentry/fsimpl/timerfd/timerfd.go
+++ b/pkg/sentry/fsimpl/timerfd/timerfd.go
@@ -19,6 +19,7 @@ import (
"sync/atomic"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
@@ -72,7 +73,7 @@ func (tfd *TimerFileDescription) Read(ctx context.Context, dst usermem.IOSequenc
}
if val := atomic.SwapUint64(&tfd.val, 0); val != 0 {
var buf [sizeofUint64]byte
- usermem.ByteOrder.PutUint64(buf[:], val)
+ hostarch.ByteOrder.PutUint64(buf[:], val)
if _, err := dst.CopyOut(ctx, buf[:]); err != nil {
// Linux does not undo consuming the number of
// expirations even if writing to userspace fails.
diff --git a/pkg/sentry/fsimpl/tmpfs/BUILD b/pkg/sentry/fsimpl/tmpfs/BUILD
index 09957c2b7..e21fddd7f 100644
--- a/pkg/sentry/fsimpl/tmpfs/BUILD
+++ b/pkg/sentry/fsimpl/tmpfs/BUILD
@@ -59,6 +59,7 @@ go_library(
"//pkg/amutex",
"//pkg/context",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
diff --git a/pkg/sentry/fsimpl/tmpfs/regular_file.go b/pkg/sentry/fsimpl/tmpfs/regular_file.go
index a6d161882..c45bddff6 100644
--- a/pkg/sentry/fsimpl/tmpfs/regular_file.go
+++ b/pkg/sentry/fsimpl/tmpfs/regular_file.go
@@ -22,6 +22,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
@@ -224,7 +225,7 @@ func (rf *regularFile) truncateLocked(newSize uint64) (bool, error) {
}
// AddMapping implements memmap.Mappable.AddMapping.
-func (rf *regularFile) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) error {
+func (rf *regularFile) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) error {
rf.mapsMu.Lock()
defer rf.mapsMu.Unlock()
rf.dataMu.RLock()
@@ -240,7 +241,7 @@ func (rf *regularFile) AddMapping(ctx context.Context, ms memmap.MappingSpace, a
pagesBefore := rf.writableMappingPages
// ar is guaranteed to be page aligned per memmap.Mappable.
- rf.writableMappingPages += uint64(ar.Length() / usermem.PageSize)
+ rf.writableMappingPages += uint64(ar.Length() / hostarch.PageSize)
if rf.writableMappingPages < pagesBefore {
panic(fmt.Sprintf("Overflow while mapping potentially writable pages pointing to a tmpfs file. Before %v, after %v", pagesBefore, rf.writableMappingPages))
@@ -251,7 +252,7 @@ func (rf *regularFile) AddMapping(ctx context.Context, ms memmap.MappingSpace, a
}
// RemoveMapping implements memmap.Mappable.RemoveMapping.
-func (rf *regularFile) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) {
+func (rf *regularFile) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) {
rf.mapsMu.Lock()
defer rf.mapsMu.Unlock()
@@ -261,7 +262,7 @@ func (rf *regularFile) RemoveMapping(ctx context.Context, ms memmap.MappingSpace
pagesBefore := rf.writableMappingPages
// ar is guaranteed to be page aligned per memmap.Mappable.
- rf.writableMappingPages -= uint64(ar.Length() / usermem.PageSize)
+ rf.writableMappingPages -= uint64(ar.Length() / hostarch.PageSize)
if rf.writableMappingPages > pagesBefore {
panic(fmt.Sprintf("Underflow while unmapping potentially writable pages pointing to a tmpfs file. Before %v, after %v", pagesBefore, rf.writableMappingPages))
@@ -270,12 +271,12 @@ func (rf *regularFile) RemoveMapping(ctx context.Context, ms memmap.MappingSpace
}
// CopyMapping implements memmap.Mappable.CopyMapping.
-func (rf *regularFile) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64, writable bool) error {
+func (rf *regularFile) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR hostarch.AddrRange, offset uint64, writable bool) error {
return rf.AddMapping(ctx, ms, dstAR, offset, writable)
}
// Translate implements memmap.Mappable.Translate.
-func (rf *regularFile) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
+func (rf *regularFile) Translate(ctx context.Context, required, optional memmap.MappableRange, at hostarch.AccessType) ([]memmap.Translation, error) {
rf.dataMu.Lock()
defer rf.dataMu.Unlock()
@@ -307,7 +308,7 @@ func (rf *regularFile) Translate(ctx context.Context, required, optional memmap.
Source: segMR,
File: rf.memFile,
Offset: seg.FileRangeOf(segMR).Start,
- Perms: usermem.AnyAccess,
+ Perms: hostarch.AnyAccess,
})
translatedEnd = segMR.End
}
@@ -487,6 +488,7 @@ func (fd *regularFileFD) Seek(ctx context.Context, offset int64, whence int32) (
// ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap.
func (fd *regularFileFD) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error {
file := fd.inode().impl.(*regularFile)
+ opts.SentryOwnedContent = true
return vfs.GenericConfigureMMap(&fd.vfsfd, file, opts)
}
@@ -539,7 +541,7 @@ func (rw *regularFileReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, er
switch {
case seg.Ok():
// Get internal mappings.
- ims, err := rw.file.memFile.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), usermem.Read)
+ ims, err := rw.file.memFile.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), hostarch.Read)
if err != nil {
return done, err
}
@@ -608,7 +610,7 @@ func (rw *regularFileReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64,
//
// See Linux, mm/filemap.c:generic_perform_write() and
// mm/shmem.c:shmem_write_begin().
- if pgstart := uint64(usermem.Addr(rw.file.size).RoundDown()); end > pgstart {
+ if pgstart := uint64(hostarch.Addr(rw.file.size).RoundDown()); end > pgstart {
end = pgstart
}
if end <= rw.off {
@@ -619,8 +621,8 @@ func (rw *regularFileReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64,
// Page-aligned mr for when we need to allocate memory. RoundUp can't
// overflow since end is an int64.
- pgstartaddr := usermem.Addr(rw.off).RoundDown()
- pgendaddr, _ := usermem.Addr(end).RoundUp()
+ pgstartaddr := hostarch.Addr(rw.off).RoundDown()
+ pgendaddr, _ := hostarch.Addr(end).RoundUp()
pgMR := memmap.MappableRange{uint64(pgstartaddr), uint64(pgendaddr)}
var (
@@ -633,7 +635,7 @@ func (rw *regularFileReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64,
switch {
case seg.Ok():
// Get internal mappings.
- ims, err := rw.file.memFile.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), usermem.Write)
+ ims, err := rw.file.memFile.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), hostarch.Write)
if err != nil {
retErr = err
goto exitLoop
diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
index 8df81f589..9ae25ce9e 100644
--- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go
+++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
@@ -36,6 +36,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/pgalloc"
@@ -43,7 +44,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/vfs/memxattr"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
- "gvisor.dev/gvisor/pkg/usermem"
)
// Name is the default filesystem name.
@@ -252,8 +252,8 @@ func (d *dentry) releaseChildrenLocked(ctx context.Context) {
// immutable
var globalStatfs = linux.Statfs{
Type: linux.TMPFS_MAGIC,
- BlockSize: usermem.PageSize,
- FragmentSize: usermem.PageSize,
+ BlockSize: hostarch.PageSize,
+ FragmentSize: hostarch.PageSize,
NameLength: linux.NAME_MAX,
// tmpfs currently does not support configurable size limits. In Linux,
@@ -263,9 +263,9 @@ var globalStatfs = linux.Statfs{
// chosen to ensure that BlockSize * Blocks does not overflow int64 (which
// applications may also handle incorrectly).
// TODO(b/29637826): allow configuring a tmpfs size and enforce it.
- Blocks: math.MaxInt64 / usermem.PageSize,
- BlocksFree: math.MaxInt64 / usermem.PageSize,
- BlocksAvailable: math.MaxInt64 / usermem.PageSize,
+ Blocks: math.MaxInt64 / hostarch.PageSize,
+ BlocksFree: math.MaxInt64 / hostarch.PageSize,
+ BlocksAvailable: math.MaxInt64 / hostarch.PageSize,
}
// dentry implements vfs.DentryImpl.
@@ -485,7 +485,7 @@ func (i *inode) statTo(stat *linux.Statx) {
linux.STATX_UID | linux.STATX_GID | linux.STATX_INO | linux.STATX_SIZE |
linux.STATX_BLOCKS | linux.STATX_ATIME | linux.STATX_CTIME |
linux.STATX_MTIME
- stat.Blksize = usermem.PageSize
+ stat.Blksize = hostarch.PageSize
stat.Nlink = atomic.LoadUint32(&i.nlink)
stat.UID = atomic.LoadUint32(&i.uid)
stat.GID = atomic.LoadUint32(&i.gid)
diff --git a/pkg/sentry/fsimpl/verity/BUILD b/pkg/sentry/fsimpl/verity/BUILD
index e265be0ee..d473a922d 100644
--- a/pkg/sentry/fsimpl/verity/BUILD
+++ b/pkg/sentry/fsimpl/verity/BUILD
@@ -14,13 +14,16 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/marshal/primitive",
"//pkg/merkletree",
"//pkg/refsvfs2",
+ "//pkg/safemem",
"//pkg/sentry/arch",
"//pkg/sentry/fs/lock",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/memmap",
"//pkg/sentry/socket/unix/transport",
"//pkg/sentry/vfs",
"//pkg/sync",
diff --git a/pkg/sentry/fsimpl/verity/filesystem.go b/pkg/sentry/fsimpl/verity/filesystem.go
index 6cb1a23e0..ca8090bbf 100644
--- a/pkg/sentry/fsimpl/verity/filesystem.go
+++ b/pkg/sentry/fsimpl/verity/filesystem.go
@@ -200,7 +200,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// contains the expected xattrs. If the file or the xattr does not
// exist, it indicates unexpected modifications to the file system.
if err == syserror.ENOENT || err == syserror.ENODATA {
- return nil, alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for %s: %v", merkleOffsetInParentXattr, childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for %s: %v", merkleOffsetInParentXattr, childPath, err))
}
if err != nil {
return nil, err
@@ -209,7 +209,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// unexpected modifications to the file system.
offset, err := strconv.Atoi(off)
if err != nil {
- return nil, alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s for %s to int: %v", merkleOffsetInParentXattr, childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s for %s to int: %v", merkleOffsetInParentXattr, childPath, err))
}
// Open parent Merkle tree file to read and verify child's hash.
@@ -223,12 +223,14 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// The parent Merkle tree file should have been created. If it's
// missing, it indicates an unexpected modification to the file system.
if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("Failed to open parent Merkle file for %s: %v", childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to open parent Merkle file for %s: %v", childPath, err))
}
if err != nil {
return nil, err
}
+ defer parentMerkleFD.DecRef(ctx)
+
// dataSize is the size of raw data for the Merkle tree. For a file,
// dataSize is the size of the whole file. For a directory, dataSize is
// the size of all its children's hashes.
@@ -241,7 +243,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// contains the expected xattrs. If the file or the xattr does not
// exist, it indicates unexpected modifications to the file system.
if err == syserror.ENOENT || err == syserror.ENODATA {
- return nil, alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for %s: %v", merkleSizeXattr, childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for %s: %v", merkleSizeXattr, childPath, err))
}
if err != nil {
return nil, err
@@ -251,7 +253,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// unexpected modifications to the file system.
parentSize, err := strconv.Atoi(dataSize)
if err != nil {
- return nil, alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s for %s to int: %v", merkleSizeXattr, childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s for %s to int: %v", merkleSizeXattr, childPath, err))
}
fdReader := FileReadWriteSeeker{
@@ -264,7 +266,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
Start: parent.lowerVD,
}, &vfs.StatOptions{})
if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("Failed to get parent stat for %s: %v", childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get parent stat for %s: %v", childPath, err))
}
if err != nil {
return nil, err
@@ -294,7 +296,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
})
parent.hashMu.RUnlock()
if err != nil && err != io.EOF {
- return nil, alertIntegrityViolation(fmt.Sprintf("Verification for %s failed: %v", childPath, err))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Verification for %s failed: %v", childPath, err))
}
// Cache child hash when it's verified the first time.
@@ -331,19 +333,21 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
Flags: linux.O_RDONLY,
})
if err == syserror.ENOENT {
- return alertIntegrityViolation(fmt.Sprintf("Failed to open merkle file for %s: %v", childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to open merkle file for %s: %v", childPath, err))
}
if err != nil {
return err
}
+ defer fd.DecRef(ctx)
+
merkleSize, err := fd.GetXattr(ctx, &vfs.GetXattrOptions{
Name: merkleSizeXattr,
Size: sizeOfStringInt32,
})
if err == syserror.ENODATA {
- return alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", merkleSizeXattr, childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", merkleSizeXattr, childPath, err))
}
if err != nil {
return err
@@ -351,7 +355,7 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
size, err := strconv.Atoi(merkleSize)
if err != nil {
- return alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s for %s to int: %v", merkleSizeXattr, childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s for %s to int: %v", merkleSizeXattr, childPath, err))
}
if d.isDir() && len(d.childrenNames) == 0 {
@@ -361,14 +365,14 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
})
if err == syserror.ENODATA {
- return alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", childrenOffsetXattr, childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", childrenOffsetXattr, childPath, err))
}
if err != nil {
return err
}
childrenOffset, err := strconv.Atoi(childrenOffString)
if err != nil {
- return alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenOffsetXattr, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenOffsetXattr, err))
}
childrenSizeString, err := fd.GetXattr(ctx, &vfs.GetXattrOptions{
@@ -377,23 +381,23 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
})
if err == syserror.ENODATA {
- return alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", childrenSizeXattr, childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", childrenSizeXattr, childPath, err))
}
if err != nil {
return err
}
childrenSize, err := strconv.Atoi(childrenSizeString)
if err != nil {
- return alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenSizeXattr, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenSizeXattr, err))
}
childrenNames := make([]byte, childrenSize)
if _, err := fd.PRead(ctx, usermem.BytesIOSequence(childrenNames), int64(childrenOffset), vfs.ReadOptions{}); err != nil {
- return alertIntegrityViolation(fmt.Sprintf("Failed to read children map for %s: %v", childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to read children map for %s: %v", childPath, err))
}
if err := json.Unmarshal(childrenNames, &d.childrenNames); err != nil {
- return alertIntegrityViolation(fmt.Sprintf("Failed to deserialize childrenNames of %s: %v", childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Failed to deserialize childrenNames of %s: %v", childPath, err))
}
}
@@ -438,7 +442,7 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
}
if _, err := merkletree.Verify(params); err != nil && err != io.EOF {
- return alertIntegrityViolation(fmt.Sprintf("Verification stat for %s failed: %v", childPath, err))
+ return fs.alertIntegrityViolation(fmt.Sprintf("Verification stat for %s failed: %v", childPath, err))
}
d.mode = uint32(stat.Mode)
d.uid = stat.UID
@@ -471,7 +475,7 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s
// The file was previously accessed. If the
// file does not exist now, it indicates an
// unexpected modification to the file system.
- return nil, alertIntegrityViolation(fmt.Sprintf("Target file %s is expected but missing", path))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Target file %s is expected but missing", path))
}
if err != nil {
return nil, err
@@ -483,7 +487,7 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s
// does not exist now, it indicates an unexpected
// modification to the file system.
if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("Expected Merkle file for target %s but none found", path))
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Expected Merkle file for target %s but none found", path))
}
if err != nil {
return nil, err
@@ -553,8 +557,8 @@ func (fs *filesystem) lookupAndVerifyLocked(ctx context.Context, parent *dentry,
}
childVD, err := parent.getLowerAt(ctx, vfsObj, name)
- if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("file %s expected but not found", parentPath+"/"+name))
+ if parent.verityEnabled() && err == syserror.ENOENT {
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("file %s expected but not found", parentPath+"/"+name))
}
if err != nil {
return nil, err
@@ -565,30 +569,31 @@ func (fs *filesystem) lookupAndVerifyLocked(ctx context.Context, parent *dentry,
defer childVD.DecRef(ctx)
childMerkleVD, err := parent.getLowerAt(ctx, vfsObj, merklePrefix+name)
- if err == syserror.ENOENT {
- if !fs.allowRuntimeEnable {
- return nil, alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", parentPath+"/"+name))
- }
- childMerkleFD, err := vfsObj.OpenAt(ctx, fs.creds, &vfs.PathOperation{
- Root: parent.lowerVD,
- Start: parent.lowerVD,
- Path: fspath.Parse(merklePrefix + name),
- }, &vfs.OpenOptions{
- Flags: linux.O_RDWR | linux.O_CREAT,
- Mode: 0644,
- })
- if err != nil {
- return nil, err
- }
- childMerkleFD.DecRef(ctx)
- childMerkleVD, err = parent.getLowerAt(ctx, vfsObj, merklePrefix+name)
- if err != nil {
+ if err != nil {
+ if err == syserror.ENOENT {
+ if parent.verityEnabled() {
+ return nil, fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", parentPath+"/"+name))
+ }
+ childMerkleFD, err := vfsObj.OpenAt(ctx, fs.creds, &vfs.PathOperation{
+ Root: parent.lowerVD,
+ Start: parent.lowerVD,
+ Path: fspath.Parse(merklePrefix + name),
+ }, &vfs.OpenOptions{
+ Flags: linux.O_RDWR | linux.O_CREAT,
+ Mode: 0644,
+ })
+ if err != nil {
+ return nil, err
+ }
+ childMerkleFD.DecRef(ctx)
+ childMerkleVD, err = parent.getLowerAt(ctx, vfsObj, merklePrefix+name)
+ if err != nil {
+ return nil, err
+ }
+ } else {
return nil, err
}
}
- if err != nil && err != syserror.ENOENT {
- return nil, err
- }
// Clear the Merkle tree file if they are to be generated at runtime.
// TODO(b/182315468): Optimize the Merkle tree generate process to
@@ -632,8 +637,6 @@ func (fs *filesystem) lookupAndVerifyLocked(ctx context.Context, parent *dentry,
childVD.IncRef()
childMerkleVD.IncRef()
- parent.IncRef()
- child.parent = parent
child.name = name
child.mode = uint32(stat.Mode)
@@ -657,6 +660,9 @@ func (fs *filesystem) lookupAndVerifyLocked(ctx context.Context, parent *dentry,
}
}
+ parent.IncRef()
+ child.parent = parent
+
return child, nil
}
@@ -855,7 +861,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
// missing, it indicates an unexpected modification to the file system.
if err != nil {
if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("File %s expected but not found", path))
+ return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("File %s expected but not found", path))
}
return nil, err
}
@@ -878,7 +884,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
// the file system.
if err != nil {
if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", path))
+ return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", path))
}
return nil, err
}
@@ -903,7 +909,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
})
if err != nil {
if err == syserror.ENOENT {
- return nil, alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", path))
+ return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", path))
}
return nil, err
}
@@ -921,7 +927,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
if err != nil {
if err == syserror.ENOENT {
parentPath, _ := d.fs.vfsfs.VirtualFilesystem().PathnameWithDeleted(ctx, d.fs.rootDentry.lowerVD, d.parent.lowerVD)
- return nil, alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", parentPath))
+ return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", parentPath))
}
return nil, err
}
diff --git a/pkg/sentry/fsimpl/verity/verity.go b/pkg/sentry/fsimpl/verity/verity.go
index 0d9b0ee2c..458c7fcb6 100644
--- a/pkg/sentry/fsimpl/verity/verity.go
+++ b/pkg/sentry/fsimpl/verity/verity.go
@@ -34,6 +34,8 @@
package verity
import (
+ "bytes"
+ "encoding/hex"
"encoding/json"
"fmt"
"math"
@@ -44,13 +46,16 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/merkletree"
"gvisor.dev/gvisor/pkg/refsvfs2"
+ "gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/arch"
fslock "gvisor.dev/gvisor/pkg/sentry/fs/lock"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
@@ -93,14 +98,18 @@ const (
)
var (
- // action specifies the action towards detected violation.
- action ViolationAction
-
// verityMu synchronizes concurrent operations that enable verity and perform
// verification checks.
verityMu sync.RWMutex
)
+// Mount option names for verityfs.
+const (
+ moptLowerPath = "lower_path"
+ moptRootHash = "root_hash"
+ moptRootName = "root_name"
+)
+
// HashAlgorithm is a type specifying the algorithm used to hash the file
// content.
type HashAlgorithm int
@@ -167,6 +176,12 @@ type filesystem struct {
// system.
alg HashAlgorithm
+ // action specifies the action towards detected violation.
+ action ViolationAction
+
+ // opts is the string mount options passed to opts.Data.
+ opts string
+
// renameMu synchronizes renaming with non-renaming operations in order
// to ensure consistent lock ordering between dentry.dirMu in different
// dentries.
@@ -189,9 +204,6 @@ type filesystem struct {
//
// +stateify savable
type InternalFilesystemOptions struct {
- // RootMerkleFileName is the name of the verity root Merkle tree file.
- RootMerkleFileName string
-
// LowerName is the name of the filesystem wrapped by verity fs.
LowerName string
@@ -199,9 +211,6 @@ type InternalFilesystemOptions struct {
// system.
Alg HashAlgorithm
- // RootHash is the root hash of the overall verity file system.
- RootHash []byte
-
// AllowRuntimeEnable specifies whether the verity file system allows
// enabling verification for files (i.e. building Merkle trees) during
// runtime.
@@ -226,8 +235,8 @@ func (FilesystemType) Release(ctx context.Context) {}
// alertIntegrityViolation alerts a violation of integrity, which usually means
// unexpected modification to the file system is detected. In ErrorOnViolation
// mode, it returns EIO, otherwise it panic.
-func alertIntegrityViolation(msg string) error {
- if action == ErrorOnViolation {
+func (fs *filesystem) alertIntegrityViolation(msg string) error {
+ if fs.action == ErrorOnViolation {
return syserror.EIO
}
panic(msg)
@@ -235,28 +244,99 @@ func alertIntegrityViolation(msg string) error {
// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
+ mopts := vfs.GenericParseMountOptions(opts.Data)
+ var rootHash []byte
+ if encodedRootHash, ok := mopts[moptRootHash]; ok {
+ delete(mopts, moptRootHash)
+ hash, err := hex.DecodeString(encodedRootHash)
+ if err != nil {
+ ctx.Warningf("verity.FilesystemType.GetFilesystem: Failed to decode root hash: %v", err)
+ return nil, nil, syserror.EINVAL
+ }
+ rootHash = hash
+ }
+ var lowerPathname string
+ if path, ok := mopts[moptLowerPath]; ok {
+ delete(mopts, moptLowerPath)
+ lowerPathname = path
+ }
+ rootName := "root"
+ if root, ok := mopts[moptRootName]; ok {
+ delete(mopts, moptRootName)
+ rootName = root
+ }
+
+ // Check for unparsed options.
+ if len(mopts) != 0 {
+ ctx.Warningf("verity.FilesystemType.GetFilesystem: unknown options: %v", mopts)
+ return nil, nil, syserror.EINVAL
+ }
+
+ // Handle internal options.
iopts, ok := opts.InternalData.(InternalFilesystemOptions)
- if !ok {
+ if len(lowerPathname) == 0 && !ok {
ctx.Warningf("verity.FilesystemType.GetFilesystem: missing verity configs")
return nil, nil, syserror.EINVAL
}
- action = iopts.Action
-
- // Mount the lower file system. The lower file system is wrapped inside
- // verity, and should not be exposed or connected.
- mopts := &vfs.MountOptions{
- GetFilesystemOptions: iopts.LowerGetFSOptions,
- InternalMount: true,
+ if len(lowerPathname) != 0 {
+ if ok {
+ ctx.Warningf("verity.FilesystemType.GetFilesystem: unexpected verity configs with specified lower path")
+ return nil, nil, syserror.EINVAL
+ }
+ iopts = InternalFilesystemOptions{
+ AllowRuntimeEnable: len(rootHash) == 0,
+ Action: ErrorOnViolation,
+ }
}
- mnt, err := vfsObj.MountDisconnected(ctx, creds, "", iopts.LowerName, mopts)
- if err != nil {
- return nil, nil, err
+
+ var lowerMount *vfs.Mount
+ var mountedLowerVD vfs.VirtualDentry
+ // Use an existing mount if lowerPath is provided.
+ if len(lowerPathname) != 0 {
+ vfsroot := vfs.RootFromContext(ctx)
+ if vfsroot.Ok() {
+ defer vfsroot.DecRef(ctx)
+ }
+ lowerPath := fspath.Parse(lowerPathname)
+ if !lowerPath.Absolute {
+ ctx.Infof("verity.FilesystemType.GetFilesystem: lower_path %q must be absolute", lowerPathname)
+ return nil, nil, syserror.EINVAL
+ }
+ var err error
+ mountedLowerVD, err = vfsObj.GetDentryAt(ctx, creds, &vfs.PathOperation{
+ Root: vfsroot,
+ Start: vfsroot,
+ Path: lowerPath,
+ FollowFinalSymlink: true,
+ }, &vfs.GetDentryOptions{
+ CheckSearchable: true,
+ })
+ if err != nil {
+ ctx.Infof("verity.FilesystemType.GetFilesystem: failed to resolve lower_path %q: %v", lowerPathname, err)
+ return nil, nil, err
+ }
+ lowerMount = mountedLowerVD.Mount()
+ defer mountedLowerVD.DecRef(ctx)
+ } else {
+ // Mount the lower file system. The lower file system is wrapped inside
+ // verity, and should not be exposed or connected.
+ mountOpts := &vfs.MountOptions{
+ GetFilesystemOptions: iopts.LowerGetFSOptions,
+ InternalMount: true,
+ }
+ mnt, err := vfsObj.MountDisconnected(ctx, creds, "", iopts.LowerName, mountOpts)
+ if err != nil {
+ return nil, nil, err
+ }
+ lowerMount = mnt
}
fs := &filesystem{
creds: creds.Fork(),
alg: iopts.Alg,
- lowerMount: mnt,
+ lowerMount: lowerMount,
+ action: iopts.Action,
+ opts: opts.Data,
allowRuntimeEnable: iopts.AllowRuntimeEnable,
}
fs.vfsfs.Init(vfsObj, &fstype, fs)
@@ -264,11 +344,11 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
// Construct the root dentry.
d := fs.newDentry()
d.refs = 1
- lowerVD := vfs.MakeVirtualDentry(mnt, mnt.Root())
+ lowerVD := vfs.MakeVirtualDentry(lowerMount, lowerMount.Root())
lowerVD.IncRef()
d.lowerVD = lowerVD
- rootMerkleName := merkleRootPrefix + iopts.RootMerkleFileName
+ rootMerkleName := merkleRootPrefix + rootName
lowerMerkleVD, err := vfsObj.GetDentryAt(ctx, fs.creds, &vfs.PathOperation{
Root: lowerVD,
@@ -309,7 +389,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
// the root Merkle file, or it's never generated.
fs.vfsfs.DecRef(ctx)
d.DecRef(ctx)
- return nil, nil, alertIntegrityViolation("Failed to find root Merkle file")
+ return nil, nil, fs.alertIntegrityViolation("Failed to find root Merkle file")
}
// Clear the Merkle tree file if they are to be generated at runtime.
@@ -348,9 +428,15 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
d.mode = uint32(stat.Mode)
d.uid = stat.UID
d.gid = stat.GID
- d.hash = make([]byte, len(iopts.RootHash))
d.childrenNames = make(map[string]struct{})
+ d.hashMu.Lock()
+ d.hash = make([]byte, len(rootHash))
+ copy(d.hash, rootHash)
+ d.hashMu.Unlock()
+
+ fs.rootDentry = d
+
if !d.isDir() {
ctx.Warningf("verity root must be a directory")
return nil, nil, syserror.EINVAL
@@ -366,7 +452,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
Size: sizeOfStringInt32,
})
if err == syserror.ENOENT || err == syserror.ENODATA {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", childrenOffsetXattr, err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", childrenOffsetXattr, err))
}
if err != nil {
return nil, nil, err
@@ -374,7 +460,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
off, err := strconv.Atoi(offString)
if err != nil {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenOffsetXattr, err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenOffsetXattr, err))
}
sizeString, err := vfsObj.GetXattrAt(ctx, creds, &vfs.PathOperation{
@@ -385,14 +471,14 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
Size: sizeOfStringInt32,
})
if err == syserror.ENOENT || err == syserror.ENODATA {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", childrenSizeXattr, err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", childrenSizeXattr, err))
}
if err != nil {
return nil, nil, err
}
size, err := strconv.Atoi(sizeString)
if err != nil {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenSizeXattr, err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", childrenSizeXattr, err))
}
lowerMerkleFD, err := vfsObj.OpenAt(ctx, fs.creds, &vfs.PathOperation{
@@ -402,19 +488,21 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
Flags: linux.O_RDONLY,
})
if err == syserror.ENOENT {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to open root Merkle file: %v", err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to open root Merkle file: %v", err))
}
if err != nil {
return nil, nil, err
}
+ defer lowerMerkleFD.DecRef(ctx)
+
childrenNames := make([]byte, size)
if _, err := lowerMerkleFD.PRead(ctx, usermem.BytesIOSequence(childrenNames), int64(off), vfs.ReadOptions{}); err != nil {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to read root children map: %v", err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to read root children map: %v", err))
}
if err := json.Unmarshal(childrenNames, &d.childrenNames); err != nil {
- return nil, nil, alertIntegrityViolation(fmt.Sprintf("Failed to deserialize childrenNames: %v", err))
+ return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to deserialize childrenNames: %v", err))
}
if err := fs.verifyStatAndChildrenLocked(ctx, d, stat); err != nil {
@@ -422,13 +510,8 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
}
}
- d.hashMu.Lock()
- copy(d.hash, iopts.RootHash)
- d.hashMu.Unlock()
d.vfsd.Init(d)
- fs.rootDentry = d
-
return &fs.vfsfs, &d.vfsd, nil
}
@@ -439,7 +522,7 @@ func (fs *filesystem) Release(ctx context.Context) {
// MountOptions implements vfs.FilesystemImpl.MountOptions.
func (fs *filesystem) MountOptions() string {
- return ""
+ return fs.opts
}
// dentry implements vfs.DentryImpl.
@@ -720,6 +803,10 @@ type fileDescription struct {
// underlying file system.
lowerFD *vfs.FileDescription
+ // lowerMappable is the memmap.Mappable corresponding to this file in the
+ // underlying file system.
+ lowerMappable memmap.Mappable
+
// merkleReader is the read-only FileDescription corresponding to the
// Merkle tree file in the underlying file system.
merkleReader *vfs.FileDescription
@@ -792,7 +879,7 @@ func (fd *fileDescription) IterDirents(ctx context.Context, cb vfs.IterDirentsCa
// Verify that the child is expected.
if dirent.Name != "." && dirent.Name != ".." {
if _, ok := fd.d.childrenNames[dirent.Name]; !ok {
- return alertIntegrityViolation(fmt.Sprintf("Unexpected children %s", dirent.Name))
+ return fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Unexpected children %s", dirent.Name))
}
}
}
@@ -806,7 +893,7 @@ func (fd *fileDescription) IterDirents(ctx context.Context, cb vfs.IterDirentsCa
// The result should contain all children plus "." and "..".
if fd.d.verityEnabled() && len(ds) != len(fd.d.childrenNames)+2 {
- return alertIntegrityViolation(fmt.Sprintf("Unexpected children number %d", len(ds)))
+ return fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Unexpected children number %d", len(ds)))
}
for fd.off < int64(len(ds)) {
@@ -978,7 +1065,7 @@ func (fd *fileDescription) enableVerity(ctx context.Context) (uintptr, error) {
// or directory other than the root, the parent Merkle tree file should
// have also been initialized.
if fd.lowerFD == nil || fd.merkleReader == nil || fd.merkleWriter == nil || (fd.parentMerkleWriter == nil && fd.d != fd.d.fs.rootDentry) {
- return 0, alertIntegrityViolation("Unexpected verity fd: missing expected underlying fds")
+ return 0, fd.d.fs.alertIntegrityViolation("Unexpected verity fd: missing expected underlying fds")
}
hash, dataSize, err := fd.generateMerkleLocked(ctx)
@@ -1033,7 +1120,7 @@ func (fd *fileDescription) enableVerity(ctx context.Context) (uintptr, error) {
}
// measureVerity returns the hash of fd, saved in verityDigest.
-func (fd *fileDescription) measureVerity(ctx context.Context, verityDigest usermem.Addr) (uintptr, error) {
+func (fd *fileDescription) measureVerity(ctx context.Context, verityDigest hostarch.Addr) (uintptr, error) {
t := kernel.TaskFromContext(ctx)
if t == nil {
return 0, syserror.EINVAL
@@ -1051,7 +1138,7 @@ func (fd *fileDescription) measureVerity(ctx context.Context, verityDigest userm
if fd.d.fs.allowRuntimeEnable {
return 0, syserror.ENODATA
}
- return 0, alertIntegrityViolation("Ioctl measureVerity: no hash found")
+ return 0, fd.d.fs.alertIntegrityViolation("Ioctl measureVerity: no hash found")
}
// The first part of VerityDigest is the metadata.
@@ -1072,11 +1159,11 @@ func (fd *fileDescription) measureVerity(ctx context.Context, verityDigest userm
}
// Now copy the root hash bytes to the memory after metadata.
- _, err := t.CopyOutBytes(usermem.Addr(uintptr(verityDigest)+linux.SizeOfDigestMetadata), fd.d.hash)
+ _, err := t.CopyOutBytes(hostarch.Addr(uintptr(verityDigest)+linux.SizeOfDigestMetadata), fd.d.hash)
return 0, err
}
-func (fd *fileDescription) verityFlags(ctx context.Context, flags usermem.Addr) (uintptr, error) {
+func (fd *fileDescription) verityFlags(ctx context.Context, flags hostarch.Addr) (uintptr, error) {
f := int32(0)
fd.d.hashMu.RLock()
@@ -1141,7 +1228,7 @@ func (fd *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, of
// contains the expected xattrs. If the xattr does not exist, it
// indicates unexpected modifications to the file system.
if err == syserror.ENODATA {
- return 0, alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", merkleSizeXattr, err))
+ return 0, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", merkleSizeXattr, err))
}
if err != nil {
return 0, err
@@ -1151,7 +1238,7 @@ func (fd *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, of
// unexpected modifications to the file system.
size, err := strconv.Atoi(dataSize)
if err != nil {
- return 0, alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", merkleSizeXattr, err))
+ return 0, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", merkleSizeXattr, err))
}
dataReader := FileReadWriteSeeker{
@@ -1184,7 +1271,7 @@ func (fd *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, of
})
fd.d.hashMu.RUnlock()
if err != nil {
- return 0, alertIntegrityViolation(fmt.Sprintf("Verification failed: %v", err))
+ return 0, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Verification failed: %v", err))
}
return n, err
}
@@ -1199,6 +1286,24 @@ func (fd *fileDescription) Write(ctx context.Context, src usermem.IOSequence, op
return 0, syserror.EROFS
}
+// ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap.
+func (fd *fileDescription) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error {
+ if err := fd.lowerFD.ConfigureMMap(ctx, opts); err != nil {
+ return err
+ }
+ fd.lowerMappable = opts.Mappable
+ if opts.MappingIdentity != nil {
+ opts.MappingIdentity.DecRef(ctx)
+ opts.MappingIdentity = nil
+ }
+
+ // Check if mmap is allowed on the lower filesystem.
+ if !opts.SentryOwnedContent {
+ return syserror.ENODEV
+ }
+ return vfs.GenericConfigureMMap(&fd.vfsfd, fd, opts)
+}
+
// LockBSD implements vfs.FileDescriptionImpl.LockBSD.
func (fd *fileDescription) LockBSD(ctx context.Context, uid fslock.UniqueID, ownerPID int32, t fslock.LockType, block fslock.Blocker) error {
return fd.lowerFD.LockBSD(ctx, ownerPID, t, block)
@@ -1224,6 +1329,115 @@ func (fd *fileDescription) TestPOSIX(ctx context.Context, uid fslock.UniqueID, t
return fd.lowerFD.TestPOSIX(ctx, uid, t, r)
}
+// Translate implements memmap.Mappable.Translate.
+func (fd *fileDescription) Translate(ctx context.Context, required, optional memmap.MappableRange, at hostarch.AccessType) ([]memmap.Translation, error) {
+ ts, err := fd.lowerMappable.Translate(ctx, required, optional, at)
+ if err != nil {
+ return ts, err
+ }
+
+ // dataSize is the size of the whole file.
+ dataSize, err := fd.merkleReader.GetXattr(ctx, &vfs.GetXattrOptions{
+ Name: merkleSizeXattr,
+ Size: sizeOfStringInt32,
+ })
+
+ // The Merkle tree file for the child should have been created and
+ // contains the expected xattrs. If the xattr does not exist, it
+ // indicates unexpected modifications to the file system.
+ if err == syserror.ENODATA {
+ return ts, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", merkleSizeXattr, err))
+ }
+ if err != nil {
+ return ts, err
+ }
+
+ // The dataSize xattr should be an integer. If it's not, it indicates
+ // unexpected modifications to the file system.
+ size, err := strconv.Atoi(dataSize)
+ if err != nil {
+ return ts, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Failed to convert xattr %s to int: %v", merkleSizeXattr, err))
+ }
+
+ merkleReader := FileReadWriteSeeker{
+ FD: fd.merkleReader,
+ Ctx: ctx,
+ }
+
+ for _, t := range ts {
+ // Content integrity relies on sentry owning the backing data. MapInternal is guaranteed
+ // to fetch sentry owned memory because we disallow verity mmaps otherwise.
+ ims, err := t.File.MapInternal(memmap.FileRange{t.Offset, t.Offset + t.Source.Length()}, hostarch.Read)
+ if err != nil {
+ return nil, err
+ }
+ dataReader := mmapReadSeeker{ims, t.Source.Start}
+ var buf bytes.Buffer
+ _, err = merkletree.Verify(&merkletree.VerifyParams{
+ Out: &buf,
+ File: &dataReader,
+ Tree: &merkleReader,
+ Size: int64(size),
+ Name: fd.d.name,
+ Mode: fd.d.mode,
+ UID: fd.d.uid,
+ GID: fd.d.gid,
+ HashAlgorithms: fd.d.fs.alg.toLinuxHashAlg(),
+ ReadOffset: int64(t.Source.Start),
+ ReadSize: int64(t.Source.Length()),
+ Expected: fd.d.hash,
+ DataAndTreeInSameFile: false,
+ })
+ if err != nil {
+ return ts, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Verification failed: %v", err))
+ }
+ }
+ return ts, err
+}
+
+// AddMapping implements memmap.Mappable.AddMapping.
+func (fd *fileDescription) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) error {
+ return fd.lowerMappable.AddMapping(ctx, ms, ar, offset, writable)
+}
+
+// RemoveMapping implements memmap.Mappable.RemoveMapping.
+func (fd *fileDescription) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar hostarch.AddrRange, offset uint64, writable bool) {
+ fd.lowerMappable.RemoveMapping(ctx, ms, ar, offset, writable)
+}
+
+// CopyMapping implements memmap.Mappable.CopyMapping.
+func (fd *fileDescription) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR hostarch.AddrRange, offset uint64, writable bool) error {
+ return fd.lowerMappable.CopyMapping(ctx, ms, srcAR, dstAR, offset, writable)
+}
+
+// InvalidateUnsavable implements memmap.Mappable.InvalidateUnsavable.
+func (fd *fileDescription) InvalidateUnsavable(context.Context) error {
+ return nil
+}
+
+// mmapReadSeeker is a helper struct used by fileDescription.Translate to pass
+// a safemem.BlockSeq pointing to the mapped region as io.ReaderAt.
+type mmapReadSeeker struct {
+ safemem.BlockSeq
+ Offset uint64
+}
+
+// ReadAt implements io.ReaderAt.ReadAt. off is the offset into the mapped file.
+func (r *mmapReadSeeker) ReadAt(p []byte, off int64) (int, error) {
+ bs := r.BlockSeq
+ // Adjust the offset into the mapped file to get the offset into the internally
+ // mapped region.
+ readOffset := off - int64(r.Offset)
+ if readOffset < 0 {
+ return 0, syserror.EINVAL
+ }
+ bs.DropFirst64(uint64(readOffset))
+ view := bs.TakeFirst64(uint64(len(p)))
+ dst := safemem.BlockSeqOf(safemem.BlockFromSafeSlice(p))
+ n, err := safemem.CopySeq(dst, view)
+ return int(n), err
+}
+
// FileReadWriteSeeker is a helper struct to pass a vfs.FileDescription as
// io.Reader/io.Writer/io.ReadSeeker/io.ReaderAt/io.WriterAt/etc.
type FileReadWriteSeeker struct {
diff --git a/pkg/sentry/fsimpl/verity/verity_test.go b/pkg/sentry/fsimpl/verity/verity_test.go
index 57bd65202..5c78a0019 100644
--- a/pkg/sentry/fsimpl/verity/verity_test.go
+++ b/pkg/sentry/fsimpl/verity/verity_test.go
@@ -89,10 +89,11 @@ func newVerityRoot(t *testing.T, hashAlg HashAlgorithm) (*vfs.VirtualFilesystem,
AllowUserMount: true,
})
+ data := "root_name=" + rootMerkleFilename
mntns, err := vfsObj.NewMountNamespace(ctx, auth.CredentialsFromContext(ctx), "", "verity", &vfs.MountOptions{
GetFilesystemOptions: vfs.GetFilesystemOptions{
+ Data: data,
InternalData: InternalFilesystemOptions{
- RootMerkleFileName: rootMerkleFilename,
LowerName: "tmpfs",
Alg: hashAlg,
AllowRuntimeEnable: true,