diff options
Diffstat (limited to 'pkg/sentry/fsimpl')
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, |