diff options
Diffstat (limited to 'pkg/sentry/fsimpl/cgroupfs')
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/BUILD | 47 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/base.go | 233 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/cgroupfs.go | 392 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/cpu.go | 56 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/cpuacct.go | 39 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/cpuset.go | 39 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/cgroupfs/memory.go | 60 |
7 files changed, 866 insertions, 0 deletions
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 +} |