diff options
42 files changed, 2352 insertions, 44 deletions
@@ -144,6 +144,7 @@ dev: $(RUNTIME_BIN) ## Installs a set of local runtimes. Requires sudo. @$(call configure_noreload,$(RUNTIME)-p,--net-raw --profile) @$(call configure_noreload,$(RUNTIME)-vfs2-d,--net-raw --debug --strace --log-packets --vfs2) @$(call configure_noreload,$(RUNTIME)-vfs2-fuse-d,--net-raw --debug --strace --log-packets --vfs2 --fuse) + @$(call configure_noreload,$(RUNTIME)-vfs2-cgroup-d,--net-raw --debug --strace --log-packets --vfs2 --cgroupfs) @$(call reload_docker) .PHONY: dev diff --git a/pkg/abi/linux/fs.go b/pkg/abi/linux/fs.go index 0d921ed6f..cad24fcc7 100644 --- a/pkg/abi/linux/fs.go +++ b/pkg/abi/linux/fs.go @@ -19,8 +19,10 @@ package linux // See linux/magic.h. const ( ANON_INODE_FS_MAGIC = 0x09041934 + CGROUP_SUPER_MAGIC = 0x27e0eb DEVPTS_SUPER_MAGIC = 0x00001cd1 EXT_SUPER_MAGIC = 0xef53 + FUSE_SUPER_MAGIC = 0x65735546 OVERLAYFS_SUPER_MAGIC = 0x794c7630 PIPEFS_MAGIC = 0x50495045 PROC_SUPER_MAGIC = 0x9fa0 @@ -29,7 +31,6 @@ const ( SYSFS_MAGIC = 0x62656572 TMPFS_MAGIC = 0x01021994 V9FS_MAGIC = 0x01021997 - FUSE_SUPER_MAGIC = 0x65735546 ) // Filesystem path limits, from uapi/linux/limits.h. diff --git a/pkg/sentry/fsimpl/cgroupfs/BUILD b/pkg/sentry/fsimpl/cgroupfs/BUILD new file mode 100644 index 000000000..48913068a --- /dev/null +++ b/pkg/sentry/fsimpl/cgroupfs/BUILD @@ -0,0 +1,47 @@ +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", + "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..360bbb17d --- /dev/null +++ b/pkg/sentry/fsimpl/cgroupfs/base.go @@ -0,0 +1,233 @@ +// 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" + "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/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.Lock() + defer d.fs.tasksMu.Unlock() + + 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.Lock() + defer d.fs.tasksMu.Unlock() + + 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 +} diff --git a/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go b/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go new file mode 100644 index 000000000..3d4005fd8 --- /dev/null +++ b/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go @@ -0,0 +1,392 @@ +// 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") + controllerMemory = kernel.CgroupControllerType("memory") +) + +var allControllers = []kernel.CgroupControllerType{controllerCPU, controllerCPUAcct, controllerCPUSet, controllerMemory} + +// SupportedMountOptions is the set of supported mount options for cgroupfs. +var SupportedMountOptions = []string{"all", "cpu", "cpuacct", "cpuset", "memory"} + +// FilesystemType implements vfs.FilesystemType. +// +// +stateify savable +type FilesystemType struct{} + +// 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.Mutex `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["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) + + for _, ty := range wantControllers { + var c controller + switch ty { + case controllerMemory: + c = newMemoryController(fs) + case controllerCPU: + c = newCPUController(fs) + case controllerCPUAcct: + c = newCPUAcctController(fs) + case controllerCPUSet: + c = newCPUSetController(fs) + default: + panic(fmt.Sprintf("Unreachable: unknown cgroup controller %q", ty)) + } + fs.controllers = append(fs.controllers, c) + } + + // 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..4641d613c --- /dev/null +++ b/pkg/sentry/fsimpl/cgroupfs/cpu.go @@ -0,0 +1,56 @@ +// 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 uint64 + cfsQuota int64 + + // CPU shares, values should be (num core * 1024). + shares uint64 +} + +var _ controller = (*cpuController)(nil) + +func newCPUController(fs *filesystem) *cpuController { + // Default values for controller parameters from Linux. + c := &cpuController{ + cfsPeriod: 100000, + cfsQuota: -1, + shares: 1024, + } + 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..0bb7f5c76 --- /dev/null +++ b/pkg/sentry/fsimpl/cgroupfs/cpuacct.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 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, _ *cgroupInode, contents map[string]kernfs.Inode) { + // This controller is currently intentionally empty. +} 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/memory.go b/pkg/sentry/fsimpl/cgroupfs/memory.go new file mode 100644 index 000000000..eaf40a753 --- /dev/null +++ b/pkg/sentry/fsimpl/cgroupfs/memory.go @@ -0,0 +1,60 @@ +// 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" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/usage" +) + +// +stateify savable +type memoryController struct { + controllerCommon +} + +var _ controller = (*memoryController)(nil) + +func newMemoryController(fs *filesystem) *memoryController { + c := &memoryController{} + 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{}) +} + +// +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/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/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/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 85909d551..b294dfd6a 100644 --- a/pkg/sentry/fsimpl/proc/task_files.go +++ b/pkg/sentry/fsimpl/proc/task_files.go @@ -1100,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/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 f0029cda6..e1a8b4409 100644 --- a/pkg/sentry/fsimpl/proc/tasks_files.go +++ b/pkg/sentry/fsimpl/proc/tasks_files.go @@ -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/kernel/BUILD b/pkg/sentry/kernel/BUILD index e9eb89378..a1ec6daab 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -141,6 +141,7 @@ go_library( srcs = [ "abstract_socket_namespace.go", "aio.go", + "cgroup.go", "context.go", "fd_table.go", "fd_table_refs.go", @@ -178,6 +179,7 @@ go_library( "task.go", "task_acct.go", "task_block.go", + "task_cgroup.go", "task_clone.go", "task_context.go", "task_exec.go", @@ -241,6 +243,7 @@ go_library( "//pkg/sentry/fs/lock", "//pkg/sentry/fs/timerfd", "//pkg/sentry/fsbridge", + "//pkg/sentry/fsimpl/kernfs", "//pkg/sentry/fsimpl/pipefs", "//pkg/sentry/fsimpl/sockfs", "//pkg/sentry/fsimpl/timerfd", diff --git a/pkg/sentry/kernel/cgroup.go b/pkg/sentry/kernel/cgroup.go new file mode 100644 index 000000000..1f1c63f37 --- /dev/null +++ b/pkg/sentry/kernel/cgroup.go @@ -0,0 +1,281 @@ +// 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 kernel + +import ( + "bytes" + "fmt" + "sort" + "sync/atomic" + + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/sync" +) + +// InvalidCgroupHierarchyID indicates an uninitialized hierarchy ID. +const InvalidCgroupHierarchyID uint32 = 0 + +// CgroupControllerType is the name of a cgroup controller. +type CgroupControllerType string + +// CgroupController is the common interface to cgroup controllers available to +// the entire sentry. The controllers themselves are defined by cgroupfs. +// +// Callers of this interface are often unable access synchronization needed to +// ensure returned values remain valid. Some of values returned from this +// interface are thus snapshots in time, and may become stale. This is ok for +// many callers like procfs. +type CgroupController interface { + // Returns the type of this cgroup controller (ex "memory", "cpu"). Returned + // value is valid for the lifetime of the controller. + Type() CgroupControllerType + + // Hierarchy returns the ID of the hierarchy this cgroup controller is + // attached to. Returned value is valid for the lifetime of the controller. + HierarchyID() uint32 + + // Filesystem returns the filesystem this controller is attached to. + // Returned value is valid for the lifetime of the controller. + Filesystem() *vfs.Filesystem + + // RootCgroup returns the root cgroup for this controller. Returned value is + // valid for the lifetime of the controller. + RootCgroup() Cgroup + + // NumCgroups returns the number of cgroups managed by this controller. + // Returned value is a snapshot in time. + NumCgroups() uint64 + + // Enabled returns whether this controller is enabled. Returned value is a + // snapshot in time. + Enabled() bool +} + +// Cgroup represents a named pointer to a cgroup in cgroupfs. When a task enters +// a cgroup, it holds a reference on the underlying dentry pointing to the +// cgroup. +// +// +stateify savable +type Cgroup struct { + *kernfs.Dentry + CgroupImpl +} + +func (c *Cgroup) decRef() { + c.Dentry.DecRef(context.Background()) +} + +// Path returns the absolute path of c, relative to its hierarchy root. +func (c *Cgroup) Path() string { + return c.FSLocalPath() +} + +// HierarchyID returns the id of the hierarchy that contains this cgroup. +func (c *Cgroup) HierarchyID() uint32 { + // Note: a cgroup is guaranteed to have at least one controller. + return c.Controllers()[0].HierarchyID() +} + +// CgroupImpl is the common interface to cgroups. +type CgroupImpl interface { + Controllers() []CgroupController + Enter(t *Task) + Leave(t *Task) +} + +// hierarchy represents a cgroupfs filesystem instance, with a unique set of +// controllers attached to it. Multiple cgroupfs mounts may reference the same +// hierarchy. +// +// +stateify savable +type hierarchy struct { + id uint32 + // These are a subset of the controllers in CgroupRegistry.controllers, + // grouped here by hierarchy for conveninent lookup. + controllers map[CgroupControllerType]CgroupController + // fs is not owned by hierarchy. The FS is responsible for unregistering the + // hierarchy on destruction, which removes this association. + fs *vfs.Filesystem +} + +func (h *hierarchy) match(ctypes []CgroupControllerType) bool { + if len(ctypes) != len(h.controllers) { + return false + } + for _, ty := range ctypes { + if _, ok := h.controllers[ty]; !ok { + return false + } + } + return true +} + +// CgroupRegistry tracks the active set of cgroup controllers on the system. +// +// +stateify savable +type CgroupRegistry struct { + // lastHierarchyID is the id of the last allocated cgroup hierarchy. Valid + // ids are from 1 to math.MaxUint32. Must be accessed through atomic ops. + // + lastHierarchyID uint32 + + mu sync.Mutex `state:"nosave"` + + // controllers is the set of currently known cgroup controllers on the + // system. Protected by mu. + // + // +checklocks:mu + controllers map[CgroupControllerType]CgroupController + + // hierarchies is the active set of cgroup hierarchies. Protected by mu. + // + // +checklocks:mu + hierarchies map[uint32]hierarchy +} + +func newCgroupRegistry() *CgroupRegistry { + return &CgroupRegistry{ + controllers: make(map[CgroupControllerType]CgroupController), + hierarchies: make(map[uint32]hierarchy), + } +} + +// nextHierarchyID returns a newly allocated, unique hierarchy ID. +func (r *CgroupRegistry) nextHierarchyID() (uint32, error) { + if hid := atomic.AddUint32(&r.lastHierarchyID, 1); hid != 0 { + return hid, nil + } + return InvalidCgroupHierarchyID, fmt.Errorf("cgroup hierarchy ID overflow") +} + +// FindHierarchy returns a cgroup filesystem containing exactly the set of +// controllers named in names. If no such FS is found, FindHierarchy return +// nil. FindHierarchy takes a reference on the returned FS, which is transferred +// to the caller. +func (r *CgroupRegistry) FindHierarchy(ctypes []CgroupControllerType) *vfs.Filesystem { + r.mu.Lock() + defer r.mu.Unlock() + + for _, h := range r.hierarchies { + if h.match(ctypes) { + h.fs.IncRef() + return h.fs + } + } + + return nil +} + +// Register registers the provided set of controllers with the registry as a new +// hierarchy. If any controller is already registered, the function returns an +// error without modifying the registry. The hierarchy can be later referenced +// by the returned id. +func (r *CgroupRegistry) Register(cs []CgroupController) (uint32, error) { + r.mu.Lock() + defer r.mu.Unlock() + + if len(cs) == 0 { + return InvalidCgroupHierarchyID, fmt.Errorf("can't register hierarchy with no controllers") + } + + for _, c := range cs { + if _, ok := r.controllers[c.Type()]; ok { + return InvalidCgroupHierarchyID, fmt.Errorf("controllers may only be mounted on a single hierarchy") + } + } + + hid, err := r.nextHierarchyID() + if err != nil { + return hid, err + } + + h := hierarchy{ + id: hid, + controllers: make(map[CgroupControllerType]CgroupController), + fs: cs[0].Filesystem(), + } + for _, c := range cs { + n := c.Type() + r.controllers[n] = c + h.controllers[n] = c + } + r.hierarchies[hid] = h + return hid, nil +} + +// Unregister removes a previously registered hierarchy from the registry. If +// the controller was not previously registered, Unregister is a no-op. +func (r *CgroupRegistry) Unregister(hid uint32) { + r.mu.Lock() + defer r.mu.Unlock() + + if h, ok := r.hierarchies[hid]; ok { + for name, _ := range h.controllers { + delete(r.controllers, name) + } + delete(r.hierarchies, hid) + } +} + +// computeInitialGroups takes a reference on each of the returned cgroups. The +// caller takes ownership of this returned reference. +func (r *CgroupRegistry) computeInitialGroups(inherit map[Cgroup]struct{}) map[Cgroup]struct{} { + r.mu.Lock() + defer r.mu.Unlock() + + ctlSet := make(map[CgroupControllerType]CgroupController) + cgset := make(map[Cgroup]struct{}) + + // Remember controllers from the inherited cgroups set... + for cg, _ := range inherit { + cg.IncRef() // Ref transferred to caller. + for _, ctl := range cg.Controllers() { + ctlSet[ctl.Type()] = ctl + cgset[cg] = struct{}{} + } + } + + // ... and add the root cgroups of all the missing controllers. + for name, ctl := range r.controllers { + if _, ok := ctlSet[name]; !ok { + cg := ctl.RootCgroup() + cg.IncRef() // Ref transferred to caller. + cgset[cg] = struct{}{} + } + } + return cgset +} + +// GenerateProcCgroups writes the contents of /proc/cgroups to buf. +func (r *CgroupRegistry) GenerateProcCgroups(buf *bytes.Buffer) { + r.mu.Lock() + entries := make([]string, 0, len(r.controllers)) + for _, c := range r.controllers { + en := 0 + if c.Enabled() { + en = 1 + } + entries = append(entries, fmt.Sprintf("%s\t%d\t%d\t%d\n", c.Type(), c.HierarchyID(), c.NumCgroups(), en)) + } + r.mu.Unlock() + + sort.Strings(entries) + fmt.Fprint(buf, "#subsys_name\thierarchy\tnum_cgroups\tenabled\n") + for _, e := range entries { + fmt.Fprint(buf, e) + } +} diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 43065b45a..9a4fd64cb 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -294,6 +294,11 @@ type Kernel struct { // YAMAPtraceScope is the current level of YAMA ptrace restrictions. YAMAPtraceScope int32 + + // cgroupRegistry contains the set of active cgroup controllers on the + // system. It is controller by cgroupfs. Nil if cgroupfs is unavailable on + // the system. + cgroupRegistry *CgroupRegistry } // InitKernelArgs holds arguments to Init. @@ -438,6 +443,8 @@ func (k *Kernel) Init(args InitKernelArgs) error { k.socketMount = socketMount k.socketsVFS2 = make(map[*vfs.FileDescription]*SocketRecord) + + k.cgroupRegistry = newCgroupRegistry() } return nil } @@ -1815,6 +1822,11 @@ func (k *Kernel) SocketMount() *vfs.Mount { return k.socketMount } +// CgroupRegistry returns the cgroup registry. +func (k *Kernel) CgroupRegistry() *CgroupRegistry { + return k.cgroupRegistry +} + // Release releases resources owned by k. // // Precondition: This should only be called after the kernel is fully @@ -1831,3 +1843,43 @@ func (k *Kernel) Release() { k.timekeeper.Destroy() k.vdso.Release(ctx) } + +// PopulateNewCgroupHierarchy moves all tasks into a newly created cgroup +// hierarchy. +// +// Precondition: root must be a new cgroup with no tasks. This implies the +// controllers for root are also new and currently manage no task, which in turn +// implies the new cgroup can be populated without migrating tasks between +// cgroups. +func (k *Kernel) PopulateNewCgroupHierarchy(root Cgroup) { + k.tasks.mu.RLock() + k.tasks.forEachTaskLocked(func(t *Task) { + if t.ExitState() != TaskExitNone { + return + } + t.mu.Lock() + t.enterCgroupLocked(root) + t.mu.Unlock() + }) + k.tasks.mu.RUnlock() +} + +// ReleaseCgroupHierarchy moves all tasks out of all cgroups belonging to the +// hierarchy with the provided id. This is intended for use during hierarchy +// teardown, as otherwise the tasks would be orphaned w.r.t to some controllers. +func (k *Kernel) ReleaseCgroupHierarchy(hid uint32) { + k.tasks.mu.RLock() + k.tasks.forEachTaskLocked(func(t *Task) { + if t.ExitState() != TaskExitNone { + return + } + t.mu.Lock() + for cg, _ := range t.cgroups { + if cg.HierarchyID() == hid { + t.leaveCgroupLocked(cg) + } + } + t.mu.Unlock() + }) + k.tasks.mu.RUnlock() +} diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index 399985039..be1371855 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -587,6 +587,12 @@ type Task struct { // // kcov is exclusive to the task goroutine. kcov *Kcov + + // cgroups is the set of cgroups this task belongs to. This may be empty if + // no cgroup controllers are enabled. Protected by mu. + // + // +checklocks:mu + cgroups map[Cgroup]struct{} } func (t *Task) savePtraceTracer() *Task { diff --git a/pkg/sentry/kernel/task_cgroup.go b/pkg/sentry/kernel/task_cgroup.go new file mode 100644 index 000000000..25d2504fa --- /dev/null +++ b/pkg/sentry/kernel/task_cgroup.go @@ -0,0 +1,138 @@ +// 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 kernel + +import ( + "bytes" + "fmt" + "sort" + "strings" + + "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/syserror" +) + +// EnterInitialCgroups moves t into an initial set of cgroups. +// +// Precondition: t isn't in any cgroups yet, t.cgs is empty. +// +// +checklocksignore parent.mu is conditionally acquired. +func (t *Task) EnterInitialCgroups(parent *Task) { + var inherit map[Cgroup]struct{} + if parent != nil { + parent.mu.Lock() + defer parent.mu.Unlock() + inherit = parent.cgroups + } + joinSet := t.k.cgroupRegistry.computeInitialGroups(inherit) + + t.mu.Lock() + defer t.mu.Unlock() + // Transfer ownership of joinSet refs to the task's cgset. + t.cgroups = joinSet + for c, _ := range t.cgroups { + // Since t isn't in any cgroup yet, we can skip the check against + // existing cgroups. + c.Enter(t) + } +} + +// EnterCgroup moves t into c. +func (t *Task) EnterCgroup(c Cgroup) error { + newControllers := make(map[CgroupControllerType]struct{}) + for _, ctl := range c.Controllers() { + newControllers[ctl.Type()] = struct{}{} + } + + t.mu.Lock() + defer t.mu.Unlock() + + for oldCG, _ := range t.cgroups { + for _, oldCtl := range oldCG.Controllers() { + if _, ok := newControllers[oldCtl.Type()]; ok { + // Already in a cgroup with the same controller as one of the + // new ones. Requires migration between cgroups. + // + // TODO(b/183137098): Implement cgroup migration. + log.Warningf("Cgroup migration is not implemented") + return syserror.EBUSY + } + } + } + + // No migration required. + t.enterCgroupLocked(c) + + return nil +} + +// +checklocks:t.mu +func (t *Task) enterCgroupLocked(c Cgroup) { + c.IncRef() + t.cgroups[c] = struct{}{} + c.Enter(t) +} + +// LeaveCgroups removes t out from all its cgroups. +func (t *Task) LeaveCgroups() { + t.mu.Lock() + defer t.mu.Unlock() + for c, _ := range t.cgroups { + t.leaveCgroupLocked(c) + } +} + +// +checklocks:t.mu +func (t *Task) leaveCgroupLocked(c Cgroup) { + c.Leave(t) + delete(t.cgroups, c) + c.decRef() +} + +// taskCgroupEntry represents a line in /proc/<pid>/cgroup, and is used to +// format a cgroup for display. +type taskCgroupEntry struct { + hierarchyID uint32 + controllers string + path string +} + +// GenerateProcTaskCgroup writes the contents of /proc/<pid>/cgroup for t to buf. +func (t *Task) GenerateProcTaskCgroup(buf *bytes.Buffer) { + t.mu.Lock() + defer t.mu.Unlock() + + cgEntries := make([]taskCgroupEntry, 0, len(t.cgroups)) + for c, _ := range t.cgroups { + ctls := c.Controllers() + ctlNames := make([]string, 0, len(ctls)) + for _, ctl := range ctls { + ctlNames = append(ctlNames, string(ctl.Type())) + } + + cgEntries = append(cgEntries, taskCgroupEntry{ + // Note: We're guaranteed to have at least one controller, and all + // controllers are guaranteed to be on the same hierarchy. + hierarchyID: ctls[0].HierarchyID(), + controllers: strings.Join(ctlNames, ","), + path: c.Path(), + }) + } + + sort.Slice(cgEntries, func(i, j int) bool { return cgEntries[i].hierarchyID > cgEntries[j].hierarchyID }) + for _, cgE := range cgEntries { + fmt.Fprintf(buf, "%d:%s:%s\n", cgE.hierarchyID, cgE.controllers, cgE.path) + } +} diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index ad59e4f60..b1af1a7ef 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -275,6 +275,10 @@ func (*runExitMain) execute(t *Task) taskRunState { t.fsContext.DecRef(t) t.fdTable.DecRef(t) + // Detach task from all cgroups. This must happen before potentially the + // last ref to the cgroupfs mount is dropped below. + t.LeaveCgroups() + t.mu.Lock() if t.mountNamespaceVFS2 != nil { t.mountNamespaceVFS2.DecRef(t) diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go index fc18b6253..32031cd70 100644 --- a/pkg/sentry/kernel/task_start.go +++ b/pkg/sentry/kernel/task_start.go @@ -151,6 +151,7 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) { rseqSignature: cfg.RSeqSignature, futexWaiter: futex.NewWaiter(), containerID: cfg.ContainerID, + cgroups: make(map[Cgroup]struct{}), } t.creds.Store(cfg.Credentials) t.endStopCond.L = &t.tg.signalHandlers.mu @@ -189,6 +190,10 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) { t.parent.children[t] = struct{}{} } + if VFS2Enabled { + t.EnterInitialCgroups(t.parent) + } + if tg.leader == nil { // New thread group. tg.leader = t diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go index 09d070ec8..77ad62445 100644 --- a/pkg/sentry/kernel/threads.go +++ b/pkg/sentry/kernel/threads.go @@ -114,6 +114,15 @@ func (ts *TaskSet) forEachThreadGroupLocked(f func(tg *ThreadGroup)) { } } +// forEachTaskLocked applies f to each Task in ts. +// +// Preconditions: ts.mu must be locked (for reading or writing). +func (ts *TaskSet) forEachTaskLocked(f func(t *Task)) { + for t := range ts.Root.tids { + f(t) + } +} + // A PIDNamespace represents a PID namespace, a bimap between thread IDs and // tasks. See the pid_namespaces(7) man page for further details. // diff --git a/pkg/sentry/vfs/file_description_impl_util.go b/pkg/sentry/vfs/file_description_impl_util.go index 1556b41a3..b87d9690a 100644 --- a/pkg/sentry/vfs/file_description_impl_util.go +++ b/pkg/sentry/vfs/file_description_impl_util.go @@ -252,6 +252,9 @@ type WritableDynamicBytesSource interface { // are backed by a bytes.Buffer that is regenerated when necessary, consistent // with Linux's fs/seq_file.c:single_open(). // +// If data additionally implements WritableDynamicBytesSource, writes are +// dispatched to the implementer. The source data is not automatically modified. +// // DynamicBytesFileDescriptionImpl.SetDataSource() must be called before first // use. // diff --git a/pkg/sentry/vfs/mount.go b/pkg/sentry/vfs/mount.go index 922f9e697..7cdab6945 100644 --- a/pkg/sentry/vfs/mount.go +++ b/pkg/sentry/vfs/mount.go @@ -970,17 +970,22 @@ func superBlockOpts(mountPath string, mnt *Mount) string { opts += "," + mopts } - // NOTE(b/147673608): If the mount is a cgroup, we also need to include - // the cgroup name in the options. For now we just read that from the - // path. + // NOTE(b/147673608): If the mount is a ramdisk-based fake cgroupfs, we also + // need to include the cgroup name in the options. For now we just read that + // from the path. Note that this is only possible when "cgroup" isn't + // registered as a valid filesystem type. // - // TODO(gvisor.dev/issue/190): Once gVisor has full cgroup support, we - // should get this value from the cgroup itself, and not rely on the - // path. + // TODO(gvisor.dev/issue/190): Once we removed fake cgroupfs support, we + // should remove this. + if cgroupfs := mnt.vfs.getFilesystemType("cgroup"); cgroupfs != nil && cgroupfs.opts.AllowUserMount { + // Real cgroupfs available. + return opts + } if mnt.fs.FilesystemType().Name() == "cgroup" { splitPath := strings.Split(mountPath, "/") cgroupType := splitPath[len(splitPath)-1] opts += "," + cgroupType } + return opts } diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD index 2b20284be..579edaa2c 100644 --- a/runsc/boot/BUILD +++ b/runsc/boot/BUILD @@ -57,6 +57,7 @@ go_library( "//pkg/sentry/fs/tmpfs", "//pkg/sentry/fs/tty", "//pkg/sentry/fs/user", + "//pkg/sentry/fsimpl/cgroupfs", "//pkg/sentry/fsimpl/devpts", "//pkg/sentry/fsimpl/devtmpfs", "//pkg/sentry/fsimpl/fuse", diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index 1ae76d7d7..05b721b28 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -400,7 +400,7 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error { // Set up the restore environment. ctx := k.SupervisorContext() - mntr := newContainerMounter(cm.l.root.spec, cm.l.root.goferFDs, cm.l.k, cm.l.mountHints, kernel.VFS2Enabled) + mntr := newContainerMounter(&cm.l.root, cm.l.k, cm.l.mountHints, kernel.VFS2Enabled) if kernel.VFS2Enabled { ctx, err = mntr.configureRestore(ctx, cm.l.root.conf) if err != nil { diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index 19ced9b0e..3c0cef6db 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -31,6 +31,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/fs/gofer" "gvisor.dev/gvisor/pkg/sentry/fs/ramfs" "gvisor.dev/gvisor/pkg/sentry/fs/user" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/cgroupfs" "gvisor.dev/gvisor/pkg/sentry/fsimpl/devpts" "gvisor.dev/gvisor/pkg/sentry/fsimpl/devtmpfs" gofervfs2 "gvisor.dev/gvisor/pkg/sentry/fsimpl/gofer" @@ -103,7 +104,7 @@ func addOverlay(ctx context.Context, conf *config.Config, lower *fs.Inode, name // compileMounts returns the supported mounts from the mount spec, adding any // mandatory mounts that are required by the OCI specification. -func compileMounts(spec *specs.Spec, vfs2Enabled bool) []specs.Mount { +func compileMounts(spec *specs.Spec, conf *config.Config, vfs2Enabled bool) []specs.Mount { // Keep track of whether proc and sys were mounted. var procMounted, sysMounted, devMounted, devptsMounted bool var mounts []specs.Mount @@ -114,6 +115,11 @@ func compileMounts(spec *specs.Spec, vfs2Enabled bool) []specs.Mount { log.Warningf("ignoring dev mount at %q", m.Destination) continue } + // Unconditionally drop any cgroupfs mounts. If requested, we'll add our + // own below. + if m.Type == cgroupfs.Name { + continue + } switch filepath.Clean(m.Destination) { case "/proc": procMounted = true @@ -132,6 +138,24 @@ func compileMounts(spec *specs.Spec, vfs2Enabled bool) []specs.Mount { // Mount proc and sys even if the user did not ask for it, as the spec // says we SHOULD. var mandatoryMounts []specs.Mount + + if conf.Cgroupfs { + mandatoryMounts = append(mandatoryMounts, specs.Mount{ + Type: tmpfsvfs2.Name, + Destination: "/sys/fs/cgroup", + }) + mandatoryMounts = append(mandatoryMounts, specs.Mount{ + Type: cgroupfs.Name, + Destination: "/sys/fs/cgroup/memory", + Options: []string{"memory"}, + }) + mandatoryMounts = append(mandatoryMounts, specs.Mount{ + Type: cgroupfs.Name, + Destination: "/sys/fs/cgroup/cpu", + Options: []string{"cpu"}, + }) + } + if !procMounted { mandatoryMounts = append(mandatoryMounts, specs.Mount{ Type: procvfs2.Name, @@ -248,6 +272,10 @@ func isSupportedMountFlag(fstype, opt string) bool { ok, err := parseMountOption(opt, tmpfsAllowedData...) return ok && err == nil } + if fstype == cgroupfs.Name { + ok, err := parseMountOption(opt, cgroupfs.SupportedMountOptions...) + return ok && err == nil + } return false } @@ -572,11 +600,11 @@ type containerMounter struct { hints *podMountHints } -func newContainerMounter(spec *specs.Spec, goferFDs []*fd.FD, k *kernel.Kernel, hints *podMountHints, vfs2Enabled bool) *containerMounter { +func newContainerMounter(info *containerInfo, k *kernel.Kernel, hints *podMountHints, vfs2Enabled bool) *containerMounter { return &containerMounter{ - root: spec.Root, - mounts: compileMounts(spec, vfs2Enabled), - fds: fdDispenser{fds: goferFDs}, + root: info.spec.Root, + mounts: compileMounts(info.spec, info.conf, vfs2Enabled), + fds: fdDispenser{fds: info.goferFDs}, k: k, hints: hints, } @@ -795,7 +823,13 @@ func (c *containerMounter) getMountNameAndOptions(conf *config.Config, m specs.M opts = p9MountData(fd, c.getMountAccessType(conf, m), conf.VFS2) // If configured, add overlay to all writable mounts. useOverlay = conf.Overlay && !mountFlags(m.Options).ReadOnly - + case cgroupfs.Name: + fsName = m.Type + var err error + opts, err = parseAndFilterOptions(m.Options, cgroupfs.SupportedMountOptions...) + if err != nil { + return "", nil, false, err + } default: log.Warningf("ignoring unknown filesystem type %q", m.Type) } diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index 774621970..95daf1f00 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -752,7 +752,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn // Setup the child container file system. l.startGoferMonitor(cid, info.goferFDs) - mntr := newContainerMounter(info.spec, info.goferFDs, l.k, l.mountHints, kernel.VFS2Enabled) + mntr := newContainerMounter(info, l.k, l.mountHints, kernel.VFS2Enabled) if root { if err := mntr.processHints(info.conf, info.procArgs.Credentials); err != nil { return nil, nil, nil, err diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go index 8b39bc59a..93c476971 100644 --- a/runsc/boot/loader_test.go +++ b/runsc/boot/loader_test.go @@ -439,7 +439,13 @@ func TestCreateMountNamespace(t *testing.T) { } defer cleanup() - mntr := newContainerMounter(&tc.spec, []*fd.FD{fd.New(sandEnd)}, nil, &podMountHints{}, false /* vfs2Enabled */) + info := containerInfo{ + conf: conf, + spec: &tc.spec, + goferFDs: []*fd.FD{fd.New(sandEnd)}, + } + + mntr := newContainerMounter(&info, nil, &podMountHints{}, false /* vfs2Enabled */) mns, err := mntr.createMountNamespace(ctx, conf) if err != nil { t.Fatalf("failed to create mount namespace: %v", err) @@ -479,7 +485,7 @@ func TestCreateMountNamespaceVFS2(t *testing.T) { defer l.Destroy() defer loaderCleanup() - mntr := newContainerMounter(l.root.spec, l.root.goferFDs, l.k, l.mountHints, true /* vfs2Enabled */) + mntr := newContainerMounter(&l.root, l.k, l.mountHints, true /* vfs2Enabled */) if err := mntr.processHints(l.root.conf, l.root.procArgs.Credentials); err != nil { t.Fatalf("failed process hints: %v", err) } @@ -702,7 +708,12 @@ func TestRestoreEnvironment(t *testing.T) { for _, ioFD := range tc.ioFDs { ioFDs = append(ioFDs, fd.New(ioFD)) } - mntr := newContainerMounter(tc.spec, ioFDs, nil, &podMountHints{}, false /* vfs2Enabled */) + info := containerInfo{ + conf: conf, + spec: tc.spec, + goferFDs: ioFDs, + } + mntr := newContainerMounter(&info, nil, &podMountHints{}, false /* vfs2Enabled */) actualRenv, err := mntr.createRestoreEnvironment(conf) if !tc.errorExpected && err != nil { t.Fatalf("could not create restore environment for test:%s", tc.name) diff --git a/runsc/boot/vfs.go b/runsc/boot/vfs.go index 5ef8cc039..9117540d5 100644 --- a/runsc/boot/vfs.go +++ b/runsc/boot/vfs.go @@ -30,6 +30,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/devices/ttydev" "gvisor.dev/gvisor/pkg/sentry/devices/tundev" "gvisor.dev/gvisor/pkg/sentry/fs/user" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/cgroupfs" "gvisor.dev/gvisor/pkg/sentry/fsimpl/devpts" "gvisor.dev/gvisor/pkg/sentry/fsimpl/devtmpfs" "gvisor.dev/gvisor/pkg/sentry/fsimpl/fuse" @@ -52,6 +53,10 @@ func registerFilesystems(k *kernel.Kernel) error { creds := auth.NewRootCredentials(k.RootUserNamespace()) vfsObj := k.VFS() + vfsObj.MustRegisterFilesystemType(cgroupfs.Name, &cgroupfs.FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ + AllowUserMount: true, + AllowUserList: true, + }) vfsObj.MustRegisterFilesystemType(devpts.Name, &devpts.FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ AllowUserList: true, // TODO(b/29356795): Users may mount this once the terminals are in a @@ -62,6 +67,10 @@ func registerFilesystems(k *kernel.Kernel) error { AllowUserMount: true, AllowUserList: true, }) + vfsObj.MustRegisterFilesystemType(fuse.Name, &fuse.FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ + AllowUserMount: true, + AllowUserList: true, + }) vfsObj.MustRegisterFilesystemType(gofer.Name, &gofer.FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ AllowUserList: true, }) @@ -81,10 +90,6 @@ func registerFilesystems(k *kernel.Kernel) error { AllowUserMount: true, AllowUserList: true, }) - vfsObj.MustRegisterFilesystemType(fuse.Name, &fuse.FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ - AllowUserMount: true, - AllowUserList: true, - }) vfsObj.MustRegisterFilesystemType(verity.Name, &verity.FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ AllowUserList: true, AllowUserMount: false, @@ -514,6 +519,13 @@ func (c *containerMounter) getMountNameAndOptionsVFS2(conf *config.Config, m *mo // If configured, add overlay to all writable mounts. useOverlay = conf.Overlay && !mountFlags(m.Options).ReadOnly + case cgroupfs.Name: + var err error + data, err = parseAndFilterOptions(m.Options, cgroupfs.SupportedMountOptions...) + if err != nil { + return "", nil, false, err + } + default: log.Warningf("ignoring unknown filesystem type %q", m.Type) return "", nil, false, nil diff --git a/runsc/config/config.go b/runsc/config/config.go index 1e5858837..0b2b97cc5 100644 --- a/runsc/config/config.go +++ b/runsc/config/config.go @@ -172,6 +172,9 @@ type Config struct { // Enables seccomp inside the sandbox. OCISeccomp bool `flag:"oci-seccomp"` + // Mounts the cgroup filesystem backed by the sentry's cgroupfs. + Cgroupfs bool `flag:"cgroupfs"` + // TestOnlyAllowRunAsCurrentUserWithoutChroot should only be used in // tests. It allows runsc to start the sandbox process as the current // user, and without chrooting the sandbox process. This can be diff --git a/runsc/config/flags.go b/runsc/config/flags.go index 1d996c841..13a1a0163 100644 --- a/runsc/config/flags.go +++ b/runsc/config/flags.go @@ -75,6 +75,7 @@ func RegisterFlags() { flag.Bool("fsgofer-host-uds", false, "allow the gofer to mount Unix Domain Sockets.") flag.Bool("vfs2", false, "enables VFSv2. This uses the new VFS layer that is faster than the previous one.") flag.Bool("fuse", false, "TEST ONLY; use while FUSE in VFSv2 is landing. This allows the use of the new experimental FUSE filesystem.") + flag.Bool("cgroupfs", false, "Automatically mount cgroupfs.") // Flags that control sandbox runtime behavior: network related. flag.Var(networkTypePtr(NetworkSandbox), "network", "specifies which network to use: sandbox (default), host, none. Using network inside the sandbox is more secure because it's isolated from the host network.") diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index ef299799e..3bd93df70 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -995,3 +995,7 @@ syscall_test( syscall_test( test = "//test/syscalls/linux:processes_test", ) + +syscall_test( + test = "//test/syscalls/linux:cgroup_test", +) diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index 043ada583..a9fa93437 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -4205,3 +4205,24 @@ cc_binary( "//test/util:test_util", ], ) + +cc_binary( + name = "cgroup_test", + testonly = 1, + srcs = ["cgroup.cc"], + linkstatic = 1, + deps = [ + "//test/util:capability_util", + "//test/util:cgroup_util", + "//test/util:file_descriptor", + "//test/util:fs_util", + "@com_google_absl//absl/strings", + gtest, + "//test/util:posix_error", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/container:flat_hash_map", + "@com_google_absl//absl/container:flat_hash_set", + ], +) diff --git a/test/syscalls/linux/cgroup.cc b/test/syscalls/linux/cgroup.cc new file mode 100644 index 000000000..3a51348ba --- /dev/null +++ b/test/syscalls/linux/cgroup.cc @@ -0,0 +1,370 @@ +// 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. + +// All tests in this file rely on being about to mount and unmount cgroupfs, +// which isn't expected to work, or be safe on a general linux system. + +#include <sys/mount.h> +#include <unistd.h> + +#include "gtest/gtest.h" +#include "absl/container/flat_hash_map.h" +#include "absl/container/flat_hash_set.h" +#include "test/util/capability_util.h" +#include "test/util/cgroup_util.h" +#include "test/util/temp_path.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { +namespace { + +using ::testing::_; +using ::testing::Gt; + +std::vector<std::string> known_controllers = {"cpu", "cpuset", "cpuacct", + "memory"}; + +bool CgroupsAvailable() { + return IsRunningOnGvisor() && !IsRunningWithVFS1() && + TEST_CHECK_NO_ERRNO_AND_VALUE(HaveCapability(CAP_SYS_ADMIN)); +} + +TEST(Cgroup, MountSucceeds) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("")); + EXPECT_NO_ERRNO(c.ContainsCallingProcess()); +} + +TEST(Cgroup, SeparateMounts) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + + for (const auto& ctl : known_controllers) { + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs(ctl)); + EXPECT_NO_ERRNO(c.ContainsCallingProcess()); + } +} + +TEST(Cgroup, AllControllersImplicit) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("")); + + absl::flat_hash_map<std::string, CgroupsEntry> cgroups_entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + for (const auto& ctl : known_controllers) { + EXPECT_TRUE(cgroups_entries.contains(ctl)) + << absl::StreamFormat("ctl=%s", ctl); + } + EXPECT_EQ(cgroups_entries.size(), known_controllers.size()); +} + +TEST(Cgroup, AllControllersExplicit) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("all")); + + absl::flat_hash_map<std::string, CgroupsEntry> cgroups_entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + for (const auto& ctl : known_controllers) { + EXPECT_TRUE(cgroups_entries.contains(ctl)) + << absl::StreamFormat("ctl=%s", ctl); + } + EXPECT_EQ(cgroups_entries.size(), known_controllers.size()); +} + +TEST(Cgroup, ProcsAndTasks) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("")); + absl::flat_hash_set<pid_t> pids = ASSERT_NO_ERRNO_AND_VALUE(c.Procs()); + absl::flat_hash_set<pid_t> tids = ASSERT_NO_ERRNO_AND_VALUE(c.Tasks()); + + EXPECT_GE(tids.size(), pids.size()) << "Found more processes than threads"; + + // Pids should be a strict subset of tids. + for (auto it = pids.begin(); it != pids.end(); ++it) { + EXPECT_TRUE(tids.contains(*it)) + << absl::StreamFormat("Have pid %d, but no such tid", *it); + } +} + +TEST(Cgroup, ControllersMustBeInUniqueHierarchy) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + // Hierarchy #1: all controllers. + Cgroup all = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("")); + // Hierarchy #2: memory. + // + // This should conflict since memory is already in hierarchy #1, and the two + // hierarchies have different sets of controllers, so this mount can't be a + // view into hierarchy #1. + EXPECT_THAT(m.MountCgroupfs("memory"), PosixErrorIs(EBUSY, _)) + << "Memory controller mounted on two hierarchies"; + EXPECT_THAT(m.MountCgroupfs("cpu"), PosixErrorIs(EBUSY, _)) + << "CPU controller mounted on two hierarchies"; +} + +TEST(Cgroup, UnmountFreesControllers) { + SKIP_IF(!CgroupsAvailable()); + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup all = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("")); + // All controllers are now attached to all's hierarchy. Attempting new mount + // with any individual controller should fail. + EXPECT_THAT(m.MountCgroupfs("memory"), PosixErrorIs(EBUSY, _)) + << "Memory controller mounted on two hierarchies"; + + // Unmount the "all" hierarchy. This should enable any controller to be + // mounted on a new hierarchy again. + ASSERT_NO_ERRNO(m.Unmount(all)); + EXPECT_NO_ERRNO(m.MountCgroupfs("memory")); + EXPECT_NO_ERRNO(m.MountCgroupfs("cpu")); +} + +TEST(Cgroup, OnlyContainsControllerSpecificFiles) { + SKIP_IF(!CgroupsAvailable()); + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup mem = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("memory")); + EXPECT_THAT(Exists(mem.Relpath("memory.usage_in_bytes")), + IsPosixErrorOkAndHolds(true)); + // CPU files shouldn't exist in memory cgroups. + EXPECT_THAT(Exists(mem.Relpath("cpu.cfs_period_us")), + IsPosixErrorOkAndHolds(false)); + EXPECT_THAT(Exists(mem.Relpath("cpu.cfs_quota_us")), + IsPosixErrorOkAndHolds(false)); + EXPECT_THAT(Exists(mem.Relpath("cpu.shares")), IsPosixErrorOkAndHolds(false)); + + Cgroup cpu = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("cpu")); + EXPECT_THAT(Exists(cpu.Relpath("cpu.cfs_period_us")), + IsPosixErrorOkAndHolds(true)); + EXPECT_THAT(Exists(cpu.Relpath("cpu.cfs_quota_us")), + IsPosixErrorOkAndHolds(true)); + EXPECT_THAT(Exists(cpu.Relpath("cpu.shares")), IsPosixErrorOkAndHolds(true)); + // Memory files shouldn't exist in cpu cgroups. + EXPECT_THAT(Exists(cpu.Relpath("memory.usage_in_bytes")), + IsPosixErrorOkAndHolds(false)); +} + +TEST(Cgroup, InvalidController) { + SKIP_IF(!CgroupsAvailable()); + + TempPath mountpoint = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); + std::string mopts = "this-controller-is-invalid"; + EXPECT_THAT( + mount("none", mountpoint.path().c_str(), "cgroup", 0, mopts.c_str()), + SyscallFailsWithErrno(EINVAL)); +} + +TEST(Cgroup, MoptAllMustBeExclusive) { + SKIP_IF(!CgroupsAvailable()); + + TempPath mountpoint = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); + std::string mopts = "all,cpu"; + EXPECT_THAT( + mount("none", mountpoint.path().c_str(), "cgroup", 0, mopts.c_str()), + SyscallFailsWithErrno(EINVAL)); +} + +TEST(MemoryCgroup, MemoryUsageInBytes) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("memory")); + EXPECT_THAT(c.ReadIntegerControlFile("memory.usage_in_bytes"), + IsPosixErrorOkAndHolds(Gt(0))); +} + +TEST(CPUCgroup, ControlFilesHaveDefaultValues) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("cpu")); + EXPECT_THAT(c.ReadIntegerControlFile("cpu.cfs_quota_us"), + IsPosixErrorOkAndHolds(-1)); + EXPECT_THAT(c.ReadIntegerControlFile("cpu.cfs_period_us"), + IsPosixErrorOkAndHolds(100000)); + EXPECT_THAT(c.ReadIntegerControlFile("cpu.shares"), + IsPosixErrorOkAndHolds(1024)); +} + +TEST(ProcCgroups, Empty) { + SKIP_IF(!CgroupsAvailable()); + + absl::flat_hash_map<std::string, CgroupsEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + // No cgroups mounted yet, we should have no entries. + EXPECT_TRUE(entries.empty()); +} + +TEST(ProcCgroups, ProcCgroupsEntries) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + + Cgroup mem = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("memory")); + absl::flat_hash_map<std::string, CgroupsEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + EXPECT_EQ(entries.size(), 1); + ASSERT_TRUE(entries.contains("memory")); + CgroupsEntry mem_e = entries["memory"]; + EXPECT_EQ(mem_e.subsys_name, "memory"); + EXPECT_GE(mem_e.hierarchy, 1); + // Expect a single root cgroup. + EXPECT_EQ(mem_e.num_cgroups, 1); + // Cgroups are currently always enabled when mounted. + EXPECT_TRUE(mem_e.enabled); + + // Add a second cgroup, and check for new entry. + + Cgroup cpu = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("cpu")); + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + EXPECT_EQ(entries.size(), 2); + EXPECT_TRUE(entries.contains("memory")); // Still have memory entry. + ASSERT_TRUE(entries.contains("cpu")); + CgroupsEntry cpu_e = entries["cpu"]; + EXPECT_EQ(cpu_e.subsys_name, "cpu"); + EXPECT_GE(cpu_e.hierarchy, 1); + EXPECT_EQ(cpu_e.num_cgroups, 1); + EXPECT_TRUE(cpu_e.enabled); + + // Separate hierarchies, since controllers were mounted separately. + EXPECT_NE(mem_e.hierarchy, cpu_e.hierarchy); +} + +TEST(ProcCgroups, UnmountRemovesEntries) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup cg = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("cpu,memory")); + absl::flat_hash_map<std::string, CgroupsEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + EXPECT_EQ(entries.size(), 2); + + ASSERT_NO_ERRNO(m.Unmount(cg)); + + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + EXPECT_TRUE(entries.empty()); +} + +TEST(ProcPIDCgroup, Empty) { + SKIP_IF(!CgroupsAvailable()); + + absl::flat_hash_map<std::string, PIDCgroupEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + EXPECT_TRUE(entries.empty()); +} + +TEST(ProcPIDCgroup, Entries) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("memory")); + + absl::flat_hash_map<std::string, PIDCgroupEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + EXPECT_EQ(entries.size(), 1); + PIDCgroupEntry mem_e = entries["memory"]; + EXPECT_GE(mem_e.hierarchy, 1); + EXPECT_EQ(mem_e.controllers, "memory"); + EXPECT_EQ(mem_e.path, "/"); + + Cgroup c1 = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("cpu")); + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + EXPECT_EQ(entries.size(), 2); + EXPECT_TRUE(entries.contains("memory")); // Still have memory entry. + PIDCgroupEntry cpu_e = entries["cpu"]; + EXPECT_GE(cpu_e.hierarchy, 1); + EXPECT_EQ(cpu_e.controllers, "cpu"); + EXPECT_EQ(cpu_e.path, "/"); + + // Separate hierarchies, since controllers were mounted separately. + EXPECT_NE(mem_e.hierarchy, cpu_e.hierarchy); +} + +TEST(ProcPIDCgroup, UnmountRemovesEntries) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup all = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("")); + + absl::flat_hash_map<std::string, PIDCgroupEntry> entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + EXPECT_GT(entries.size(), 0); + + ASSERT_NO_ERRNO(m.Unmount(all)); + + entries = ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + EXPECT_TRUE(entries.empty()); +} + +TEST(ProcCgroup, PIDCgroupMatchesCgroups) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("memory")); + Cgroup c1 = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("cpu")); + + absl::flat_hash_map<std::string, CgroupsEntry> cgroups_entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + absl::flat_hash_map<std::string, PIDCgroupEntry> pid_entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + + CgroupsEntry cgroup_mem = cgroups_entries["memory"]; + PIDCgroupEntry pid_mem = pid_entries["memory"]; + + EXPECT_EQ(cgroup_mem.hierarchy, pid_mem.hierarchy); + + CgroupsEntry cgroup_cpu = cgroups_entries["cpu"]; + PIDCgroupEntry pid_cpu = pid_entries["cpu"]; + + EXPECT_EQ(cgroup_cpu.hierarchy, pid_cpu.hierarchy); + EXPECT_NE(cgroup_mem.hierarchy, cgroup_cpu.hierarchy); + EXPECT_NE(pid_mem.hierarchy, pid_cpu.hierarchy); +} + +TEST(ProcCgroup, MultiControllerHierarchy) { + SKIP_IF(!CgroupsAvailable()); + + Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir())); + Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("memory,cpu")); + + absl::flat_hash_map<std::string, CgroupsEntry> cgroups_entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcCgroupsEntries()); + + CgroupsEntry mem_e = cgroups_entries["memory"]; + CgroupsEntry cpu_e = cgroups_entries["cpu"]; + + // Both controllers should have the same hierarchy ID. + EXPECT_EQ(mem_e.hierarchy, cpu_e.hierarchy); + + absl::flat_hash_map<std::string, PIDCgroupEntry> pid_entries = + ASSERT_NO_ERRNO_AND_VALUE(ProcPIDCgroupEntries(getpid())); + + // Expecting an entry listing both controllers, that matches the previous + // hierarchy ID. Note that the controllers are listed in alphabetical order. + PIDCgroupEntry pid_e = pid_entries["cpu,memory"]; + EXPECT_EQ(pid_e.hierarchy, mem_e.hierarchy); +} + +} // namespace +} // namespace testing +} // namespace gvisor diff --git a/test/util/BUILD b/test/util/BUILD index e561f3daa..383de00ed 100644 --- a/test/util/BUILD +++ b/test/util/BUILD @@ -94,6 +94,7 @@ cc_library( ":file_descriptor", ":posix_error", "@com_google_absl//absl/strings", + "@com_google_absl//absl/time", gtest, ], ) @@ -368,3 +369,20 @@ cc_library( testonly = 1, hdrs = ["temp_umask.h"], ) + +cc_library( + name = "cgroup_util", + testonly = 1, + srcs = ["cgroup_util.cc"], + hdrs = ["cgroup_util.h"], + deps = [ + ":cleanup", + ":fs_util", + ":mount_util", + ":posix_error", + ":temp_path", + "@com_google_absl//absl/container:flat_hash_map", + "@com_google_absl//absl/container:flat_hash_set", + "@com_google_absl//absl/strings", + ], +) diff --git a/test/util/cgroup_util.cc b/test/util/cgroup_util.cc new file mode 100644 index 000000000..65d9c4986 --- /dev/null +++ b/test/util/cgroup_util.cc @@ -0,0 +1,223 @@ +// 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. + +#include "test/util/cgroup_util.h" + +#include <sys/syscall.h> +#include <unistd.h> + +#include "absl/strings/str_split.h" +#include "test/util/fs_util.h" +#include "test/util/mount_util.h" + +namespace gvisor { +namespace testing { + +Cgroup::Cgroup(std::string path) : cgroup_path_(path) { + id_ = ++Cgroup::next_id_; + std::cerr << absl::StreamFormat("[cg#%d] <= %s", id_, cgroup_path_) + << std::endl; +} + +PosixErrorOr<std::string> Cgroup::ReadControlFile( + absl::string_view name) const { + std::string buf; + RETURN_IF_ERRNO(GetContents(Relpath(name), &buf)); + + const std::string alias_path = absl::StrFormat("[cg#%d]/%s", id_, name); + std::cerr << absl::StreamFormat("<contents of %s>", alias_path) << std::endl; + std::cerr << buf; + std::cerr << absl::StreamFormat("<end of %s>", alias_path) << std::endl; + + return buf; +} + +PosixErrorOr<int64_t> Cgroup::ReadIntegerControlFile( + absl::string_view name) const { + ASSIGN_OR_RETURN_ERRNO(const std::string buf, ReadControlFile(name)); + ASSIGN_OR_RETURN_ERRNO(const int64_t val, Atoi<int64_t>(buf)); + return val; +} + +PosixErrorOr<absl::flat_hash_set<pid_t>> Cgroup::Procs() const { + ASSIGN_OR_RETURN_ERRNO(std::string buf, ReadControlFile("cgroup.procs")); + return ParsePIDList(buf); +} + +PosixErrorOr<absl::flat_hash_set<pid_t>> Cgroup::Tasks() const { + ASSIGN_OR_RETURN_ERRNO(std::string buf, ReadControlFile("tasks")); + return ParsePIDList(buf); +} + +PosixError Cgroup::ContainsCallingProcess() const { + ASSIGN_OR_RETURN_ERRNO(const absl::flat_hash_set<pid_t> procs, Procs()); + ASSIGN_OR_RETURN_ERRNO(const absl::flat_hash_set<pid_t> tasks, Tasks()); + const pid_t pid = getpid(); + const pid_t tid = syscall(SYS_gettid); + if (!procs.contains(pid)) { + return PosixError( + ENOENT, absl::StrFormat("Cgroup doesn't contain process %d", pid)); + } + if (!tasks.contains(tid)) { + return PosixError(ENOENT, + absl::StrFormat("Cgroup doesn't contain task %d", tid)); + } + return NoError(); +} + +PosixErrorOr<absl::flat_hash_set<pid_t>> Cgroup::ParsePIDList( + absl::string_view data) const { + absl::flat_hash_set<pid_t> res; + std::vector<absl::string_view> lines = absl::StrSplit(data, '\n'); + for (const std::string_view& line : lines) { + if (line.empty()) { + continue; + } + ASSIGN_OR_RETURN_ERRNO(const int32_t pid, Atoi<int32_t>(line)); + res.insert(static_cast<pid_t>(pid)); + } + return res; +} + +int64_t Cgroup::next_id_ = 0; + +PosixErrorOr<Cgroup> Mounter::MountCgroupfs(std::string mopts) { + ASSIGN_OR_RETURN_ERRNO(TempPath mountpoint, + TempPath::CreateDirIn(root_.path())); + ASSIGN_OR_RETURN_ERRNO( + Cleanup mount, Mount("none", mountpoint.path(), "cgroup", 0, mopts, 0)); + const std::string mountpath = mountpoint.path(); + std::cerr << absl::StreamFormat( + "Mount(\"none\", \"%s\", \"cgroup\", 0, \"%s\", 0) => OK", + mountpath, mopts) + << std::endl; + Cgroup cg = Cgroup(mountpath); + mountpoints_[cg.id()] = std::move(mountpoint); + mounts_[cg.id()] = std::move(mount); + return cg; +} + +PosixError Mounter::Unmount(const Cgroup& c) { + auto mount = mounts_.find(c.id()); + auto mountpoint = mountpoints_.find(c.id()); + + if (mount == mounts_.end() || mountpoint == mountpoints_.end()) { + return PosixError( + ESRCH, absl::StrFormat("No mount found for cgroupfs containing cg#%d", + c.id())); + } + + std::cerr << absl::StreamFormat("Unmount([cg#%d])", c.id()) << std::endl; + + // Simply delete the entries, their destructors will unmount and delete the + // mountpoint. Note the order is important to avoid errors: mount then + // mountpoint. + mounts_.erase(mount); + mountpoints_.erase(mountpoint); + + return NoError(); +} + +constexpr char kProcCgroupsHeader[] = + "#subsys_name\thierarchy\tnum_cgroups\tenabled"; + +PosixErrorOr<absl::flat_hash_map<std::string, CgroupsEntry>> +ProcCgroupsEntries() { + std::string content; + RETURN_IF_ERRNO(GetContents("/proc/cgroups", &content)); + + bool found_header = false; + absl::flat_hash_map<std::string, CgroupsEntry> entries; + std::vector<std::string> lines = absl::StrSplit(content, '\n'); + std::cerr << "<contents of /proc/cgroups>" << std::endl; + for (const std::string& line : lines) { + std::cerr << line << std::endl; + + if (!found_header) { + EXPECT_EQ(line, kProcCgroupsHeader); + found_header = true; + continue; + } + if (line.empty()) { + continue; + } + + // Parse a single entry from /proc/cgroups. + // + // Example entries, fields are tab separated in the real file: + // + // #subsys_name hierarchy num_cgroups enabled + // cpuset 12 35 1 + // cpu 3 222 1 + // ^ ^ ^ ^ + // 0 1 2 3 + + CgroupsEntry entry; + std::vector<std::string> fields = + StrSplit(line, absl::ByAnyChar(": \t"), absl::SkipEmpty()); + + entry.subsys_name = fields[0]; + ASSIGN_OR_RETURN_ERRNO(entry.hierarchy, Atoi<uint32_t>(fields[1])); + ASSIGN_OR_RETURN_ERRNO(entry.num_cgroups, Atoi<uint64_t>(fields[2])); + ASSIGN_OR_RETURN_ERRNO(const int enabled, Atoi<int>(fields[3])); + entry.enabled = enabled != 0; + + entries[entry.subsys_name] = entry; + } + std::cerr << "<end of /proc/cgroups>" << std::endl; + + return entries; +} + +PosixErrorOr<absl::flat_hash_map<std::string, PIDCgroupEntry>> +ProcPIDCgroupEntries(pid_t pid) { + const std::string path = absl::StrFormat("/proc/%d/cgroup", pid); + std::string content; + RETURN_IF_ERRNO(GetContents(path, &content)); + + absl::flat_hash_map<std::string, PIDCgroupEntry> entries; + std::vector<std::string> lines = absl::StrSplit(content, '\n'); + + std::cerr << absl::StreamFormat("<contents of %s>", path) << std::endl; + for (const std::string& line : lines) { + std::cerr << line << std::endl; + + if (line.empty()) { + continue; + } + + // Parse a single entry from /proc/<pid>/cgroup. + // + // Example entries: + // + // 2:cpu:/path/to/cgroup + // 1:memory:/ + + PIDCgroupEntry entry; + std::vector<std::string> fields = + absl::StrSplit(line, absl::ByChar(':'), absl::SkipEmpty()); + + ASSIGN_OR_RETURN_ERRNO(entry.hierarchy, Atoi<uint32_t>(fields[0])); + entry.controllers = fields[1]; + entry.path = fields[2]; + + entries[entry.controllers] = entry; + } + std::cerr << absl::StreamFormat("<end of %s>", path) << std::endl; + + return entries; +} + +} // namespace testing +} // namespace gvisor diff --git a/test/util/cgroup_util.h b/test/util/cgroup_util.h new file mode 100644 index 000000000..b049559df --- /dev/null +++ b/test/util/cgroup_util.h @@ -0,0 +1,111 @@ +// 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. + +#ifndef GVISOR_TEST_UTIL_CGROUP_UTIL_H_ +#define GVISOR_TEST_UTIL_CGROUP_UTIL_H_ + +#include <unistd.h> + +#include "absl/container/flat_hash_map.h" +#include "absl/container/flat_hash_set.h" +#include "absl/strings/string_view.h" +#include "test/util/cleanup.h" +#include "test/util/fs_util.h" +#include "test/util/temp_path.h" + +namespace gvisor { +namespace testing { + +// Cgroup represents a cgroup directory on a mounted cgroupfs. +class Cgroup { + public: + Cgroup(std::string path); + + uint64_t id() const { return id_; } + + std::string Relpath(absl::string_view leaf) const { + return JoinPath(cgroup_path_, leaf); + } + + // Returns the contents of a cgroup control file with the given name. + PosixErrorOr<std::string> ReadControlFile(absl::string_view name) const; + + // Reads the contents of a cgroup control with the given name, and attempts + // to parse it as an integer. + PosixErrorOr<int64_t> ReadIntegerControlFile(absl::string_view name) const; + + // Returns the thread ids of the leaders of thread groups managed by this + // cgroup. + PosixErrorOr<absl::flat_hash_set<pid_t>> Procs() const; + + PosixErrorOr<absl::flat_hash_set<pid_t>> Tasks() const; + + // ContainsCallingProcess checks whether the calling process is part of the + PosixError ContainsCallingProcess() const; + + private: + PosixErrorOr<absl::flat_hash_set<pid_t>> ParsePIDList( + absl::string_view data) const; + + static int64_t next_id_; + int64_t id_; + const std::string cgroup_path_; +}; + +// Mounter is a utility for creating cgroupfs mounts. It automatically manages +// the lifetime of created mounts. +class Mounter { + public: + Mounter(TempPath root) : root_(std::move(root)) {} + + PosixErrorOr<Cgroup> MountCgroupfs(std::string mopts); + + PosixError Unmount(const Cgroup& c); + + private: + // The destruction order of these members avoids errors during cleanup. We + // first unmount (by executing the mounts_ cleanups), then delete the + // mountpoint subdirs, then delete the root. + TempPath root_; + absl::flat_hash_map<int64_t, TempPath> mountpoints_; + absl::flat_hash_map<int64_t, Cleanup> mounts_; +}; + +// Represents a line from /proc/cgroups. +struct CgroupsEntry { + std::string subsys_name; + uint32_t hierarchy; + uint64_t num_cgroups; + bool enabled; +}; + +// Returns a parsed representation of /proc/cgroups. +PosixErrorOr<absl::flat_hash_map<std::string, CgroupsEntry>> +ProcCgroupsEntries(); + +// Represents a line from /proc/<pid>/cgroup. +struct PIDCgroupEntry { + uint32_t hierarchy; + std::string controllers; + std::string path; +}; + +// Returns a parsed representation of /proc/<pid>/cgroup. +PosixErrorOr<absl::flat_hash_map<std::string, PIDCgroupEntry>> +ProcPIDCgroupEntries(pid_t pid); + +} // namespace testing +} // namespace gvisor + +#endif // GVISOR_TEST_UTIL_CGROUP_UTIL_H_ diff --git a/test/util/fs_util.cc b/test/util/fs_util.cc index 5f1ce0d8a..483ae848d 100644 --- a/test/util/fs_util.cc +++ b/test/util/fs_util.cc @@ -28,6 +28,8 @@ #include "absl/strings/str_cat.h" #include "absl/strings/str_split.h" #include "absl/strings/string_view.h" +#include "absl/time/clock.h" +#include "absl/time/time.h" #include "test/util/cleanup.h" #include "test/util/file_descriptor.h" #include "test/util/posix_error.h" @@ -366,6 +368,48 @@ PosixErrorOr<std::vector<std::string>> ListDir(absl::string_view abspath, return files; } +PosixError DirContains(absl::string_view path, + const std::vector<std::string>& expect, + const std::vector<std::string>& exclude) { + ASSIGN_OR_RETURN_ERRNO(auto listing, ListDir(path, false)); + + for (auto& expected_entry : expect) { + auto cursor = std::find(listing.begin(), listing.end(), expected_entry); + if (cursor == listing.end()) { + return PosixError(ENOENT, absl::StrFormat("Failed to find '%s' in '%s'", + expected_entry, path)); + } + } + for (auto& excluded_entry : exclude) { + auto cursor = std::find(listing.begin(), listing.end(), excluded_entry); + if (cursor != listing.end()) { + return PosixError(ENOENT, absl::StrCat("File '", excluded_entry, + "' found in path '", path, "'")); + } + } + return NoError(); +} + +PosixError EventuallyDirContains(absl::string_view path, + const std::vector<std::string>& expect, + const std::vector<std::string>& exclude) { + constexpr int kRetryCount = 100; + const absl::Duration kRetryDelay = absl::Milliseconds(100); + + for (int i = 0; i < kRetryCount; ++i) { + auto res = DirContains(path, expect, exclude); + if (res.ok()) { + return res; + } + if (i < kRetryCount - 1) { + // Sleep if this isn't the final iteration. + absl::SleepFor(kRetryDelay); + } + } + return PosixError(ETIMEDOUT, + "Timed out while waiting for directory to contain files "); +} + PosixError RecursivelyDelete(absl::string_view path, int* undeleted_dirs, int* undeleted_files) { ASSIGN_OR_RETURN_ERRNO(bool exists, Exists(path)); diff --git a/test/util/fs_util.h b/test/util/fs_util.h index 2190c3bca..bb2d1d3c8 100644 --- a/test/util/fs_util.h +++ b/test/util/fs_util.h @@ -129,6 +129,18 @@ PosixError WalkTree( PosixErrorOr<std::vector<std::string>> ListDir(absl::string_view abspath, bool skipdots); +// Check that a directory contains children nodes named in expect, and does not +// contain any children nodes named in exclude. +PosixError DirContains(absl::string_view path, + const std::vector<std::string>& expect, + const std::vector<std::string>& exclude); + +// Same as DirContains, but adds a retry. Suitable for checking a directory +// being modified asynchronously. +PosixError EventuallyDirContains(absl::string_view path, + const std::vector<std::string>& expect, + const std::vector<std::string>& exclude); + // Attempt to recursively delete a directory or file. Returns an error and // the number of undeleted directories and files. If either // undeleted_dirs or undeleted_files is nullptr then it will not be used. |