summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--Makefile1
-rw-r--r--pkg/abi/linux/fs.go3
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/BUILD47
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/base.go233
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cgroupfs.go392
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpu.go56
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpuacct.go39
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpuset.go39
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/memory.go60
-rw-r--r--pkg/sentry/fsimpl/kernfs/dynamic_bytes_file.go10
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs.go10
-rw-r--r--pkg/sentry/fsimpl/proc/filesystem.go6
-rw-r--r--pkg/sentry/fsimpl/proc/task.go23
-rw-r--r--pkg/sentry/fsimpl/proc/task_files.go29
-rw-r--r--pkg/sentry/fsimpl/proc/tasks.go19
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_files.go16
-rw-r--r--pkg/sentry/kernel/BUILD3
-rw-r--r--pkg/sentry/kernel/cgroup.go281
-rw-r--r--pkg/sentry/kernel/kernel.go52
-rw-r--r--pkg/sentry/kernel/task.go6
-rw-r--r--pkg/sentry/kernel/task_cgroup.go138
-rw-r--r--pkg/sentry/kernel/task_exit.go4
-rw-r--r--pkg/sentry/kernel/task_start.go5
-rw-r--r--pkg/sentry/kernel/threads.go9
-rw-r--r--pkg/sentry/vfs/file_description_impl_util.go3
-rw-r--r--pkg/sentry/vfs/mount.go17
-rw-r--r--runsc/boot/BUILD1
-rw-r--r--runsc/boot/controller.go2
-rw-r--r--runsc/boot/fs.go46
-rw-r--r--runsc/boot/loader.go2
-rw-r--r--runsc/boot/loader_test.go17
-rw-r--r--runsc/boot/vfs.go20
-rw-r--r--runsc/config/config.go3
-rw-r--r--runsc/config/flags.go1
-rw-r--r--test/syscalls/BUILD4
-rw-r--r--test/syscalls/linux/BUILD21
-rw-r--r--test/syscalls/linux/cgroup.cc370
-rw-r--r--test/util/BUILD18
-rw-r--r--test/util/cgroup_util.cc223
-rw-r--r--test/util/cgroup_util.h111
-rw-r--r--test/util/fs_util.cc44
-rw-r--r--test/util/fs_util.h12
42 files changed, 2352 insertions, 44 deletions
diff --git a/Makefile b/Makefile
index 56c912713..ea0674f77 100644
--- a/Makefile
+++ b/Makefile
@@ -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.