summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fsimpl
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2021-04-03 04:15:14 +0000
committergVisor bot <gvisor-bot@google.com>2021-04-03 04:15:14 +0000
commit0e1d141ffcf7878ff60a3ed1f1c696ecfa8d099e (patch)
tree9755d118b4801cb9743df8617128888ab06470e4 /pkg/sentry/fsimpl
parent3dee9c57344555ab56d44c18c2b9f7c667d2d593 (diff)
parent932c8abd0f739bec295ff62cf8fce3dcb7e2d866 (diff)
Merge release-20210322.0-38-g932c8abd0 (automated)
Diffstat (limited to 'pkg/sentry/fsimpl')
-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/cgroupfs_state_autogen.go473
-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/dir_refs.go132
-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/proc_state_autogen.go61
-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
16 files changed, 1573 insertions, 25 deletions
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/cgroupfs_state_autogen.go b/pkg/sentry/fsimpl/cgroupfs/cgroupfs_state_autogen.go
new file mode 100644
index 000000000..9e2e9de36
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/cgroupfs_state_autogen.go
@@ -0,0 +1,473 @@
+// automatically generated by stateify.
+
+package cgroupfs
+
+import (
+ "gvisor.dev/gvisor/pkg/state"
+)
+
+func (c *controllerCommon) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.controllerCommon"
+}
+
+func (c *controllerCommon) StateFields() []string {
+ return []string{
+ "ty",
+ "fs",
+ }
+}
+
+func (c *controllerCommon) beforeSave() {}
+
+// +checklocksignore
+func (c *controllerCommon) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.ty)
+ stateSinkObject.Save(1, &c.fs)
+}
+
+func (c *controllerCommon) afterLoad() {}
+
+// +checklocksignore
+func (c *controllerCommon) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.ty)
+ stateSourceObject.Load(1, &c.fs)
+}
+
+func (c *cgroupInode) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.cgroupInode"
+}
+
+func (c *cgroupInode) StateFields() []string {
+ return []string{
+ "dir",
+ "fs",
+ "ts",
+ }
+}
+
+func (c *cgroupInode) beforeSave() {}
+
+// +checklocksignore
+func (c *cgroupInode) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.dir)
+ stateSinkObject.Save(1, &c.fs)
+ stateSinkObject.Save(2, &c.ts)
+}
+
+func (c *cgroupInode) afterLoad() {}
+
+// +checklocksignore
+func (c *cgroupInode) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.dir)
+ stateSourceObject.Load(1, &c.fs)
+ stateSourceObject.Load(2, &c.ts)
+}
+
+func (d *cgroupProcsData) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.cgroupProcsData"
+}
+
+func (d *cgroupProcsData) StateFields() []string {
+ return []string{
+ "cgroupInode",
+ }
+}
+
+func (d *cgroupProcsData) beforeSave() {}
+
+// +checklocksignore
+func (d *cgroupProcsData) StateSave(stateSinkObject state.Sink) {
+ d.beforeSave()
+ stateSinkObject.Save(0, &d.cgroupInode)
+}
+
+func (d *cgroupProcsData) afterLoad() {}
+
+// +checklocksignore
+func (d *cgroupProcsData) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &d.cgroupInode)
+}
+
+func (d *tasksData) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.tasksData"
+}
+
+func (d *tasksData) StateFields() []string {
+ return []string{
+ "cgroupInode",
+ }
+}
+
+func (d *tasksData) beforeSave() {}
+
+// +checklocksignore
+func (d *tasksData) StateSave(stateSinkObject state.Sink) {
+ d.beforeSave()
+ stateSinkObject.Save(0, &d.cgroupInode)
+}
+
+func (d *tasksData) afterLoad() {}
+
+// +checklocksignore
+func (d *tasksData) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &d.cgroupInode)
+}
+
+func (fsType *FilesystemType) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.FilesystemType"
+}
+
+func (fsType *FilesystemType) StateFields() []string {
+ return []string{}
+}
+
+func (fsType *FilesystemType) beforeSave() {}
+
+// +checklocksignore
+func (fsType *FilesystemType) StateSave(stateSinkObject state.Sink) {
+ fsType.beforeSave()
+}
+
+func (fsType *FilesystemType) afterLoad() {}
+
+// +checklocksignore
+func (fsType *FilesystemType) StateLoad(stateSourceObject state.Source) {
+}
+
+func (fs *filesystem) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.filesystem"
+}
+
+func (fs *filesystem) StateFields() []string {
+ return []string{
+ "Filesystem",
+ "devMinor",
+ "hierarchyID",
+ "controllers",
+ "kcontrollers",
+ "numCgroups",
+ "root",
+ }
+}
+
+func (fs *filesystem) beforeSave() {}
+
+// +checklocksignore
+func (fs *filesystem) StateSave(stateSinkObject state.Sink) {
+ fs.beforeSave()
+ stateSinkObject.Save(0, &fs.Filesystem)
+ stateSinkObject.Save(1, &fs.devMinor)
+ stateSinkObject.Save(2, &fs.hierarchyID)
+ stateSinkObject.Save(3, &fs.controllers)
+ stateSinkObject.Save(4, &fs.kcontrollers)
+ stateSinkObject.Save(5, &fs.numCgroups)
+ stateSinkObject.Save(6, &fs.root)
+}
+
+func (fs *filesystem) afterLoad() {}
+
+// +checklocksignore
+func (fs *filesystem) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &fs.Filesystem)
+ stateSourceObject.Load(1, &fs.devMinor)
+ stateSourceObject.Load(2, &fs.hierarchyID)
+ stateSourceObject.Load(3, &fs.controllers)
+ stateSourceObject.Load(4, &fs.kcontrollers)
+ stateSourceObject.Load(5, &fs.numCgroups)
+ stateSourceObject.Load(6, &fs.root)
+}
+
+func (i *implStatFS) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.implStatFS"
+}
+
+func (i *implStatFS) StateFields() []string {
+ return []string{}
+}
+
+func (i *implStatFS) beforeSave() {}
+
+// +checklocksignore
+func (i *implStatFS) StateSave(stateSinkObject state.Sink) {
+ i.beforeSave()
+}
+
+func (i *implStatFS) afterLoad() {}
+
+// +checklocksignore
+func (i *implStatFS) StateLoad(stateSourceObject state.Source) {
+}
+
+func (d *dir) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.dir"
+}
+
+func (d *dir) StateFields() []string {
+ return []string{
+ "dirRefs",
+ "InodeAlwaysValid",
+ "InodeAttrs",
+ "InodeNotSymlink",
+ "InodeDirectoryNoNewChildren",
+ "OrderedChildren",
+ "implStatFS",
+ "locks",
+ }
+}
+
+func (d *dir) beforeSave() {}
+
+// +checklocksignore
+func (d *dir) StateSave(stateSinkObject state.Sink) {
+ d.beforeSave()
+ stateSinkObject.Save(0, &d.dirRefs)
+ stateSinkObject.Save(1, &d.InodeAlwaysValid)
+ stateSinkObject.Save(2, &d.InodeAttrs)
+ stateSinkObject.Save(3, &d.InodeNotSymlink)
+ stateSinkObject.Save(4, &d.InodeDirectoryNoNewChildren)
+ stateSinkObject.Save(5, &d.OrderedChildren)
+ stateSinkObject.Save(6, &d.implStatFS)
+ stateSinkObject.Save(7, &d.locks)
+}
+
+func (d *dir) afterLoad() {}
+
+// +checklocksignore
+func (d *dir) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &d.dirRefs)
+ stateSourceObject.Load(1, &d.InodeAlwaysValid)
+ stateSourceObject.Load(2, &d.InodeAttrs)
+ stateSourceObject.Load(3, &d.InodeNotSymlink)
+ stateSourceObject.Load(4, &d.InodeDirectoryNoNewChildren)
+ stateSourceObject.Load(5, &d.OrderedChildren)
+ stateSourceObject.Load(6, &d.implStatFS)
+ stateSourceObject.Load(7, &d.locks)
+}
+
+func (c *controllerFile) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.controllerFile"
+}
+
+func (c *controllerFile) StateFields() []string {
+ return []string{
+ "DynamicBytesFile",
+ }
+}
+
+func (c *controllerFile) beforeSave() {}
+
+// +checklocksignore
+func (c *controllerFile) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.DynamicBytesFile)
+}
+
+func (c *controllerFile) afterLoad() {}
+
+// +checklocksignore
+func (c *controllerFile) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.DynamicBytesFile)
+}
+
+func (s *staticControllerFile) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.staticControllerFile"
+}
+
+func (s *staticControllerFile) StateFields() []string {
+ return []string{
+ "DynamicBytesFile",
+ "StaticData",
+ }
+}
+
+func (s *staticControllerFile) beforeSave() {}
+
+// +checklocksignore
+func (s *staticControllerFile) StateSave(stateSinkObject state.Sink) {
+ s.beforeSave()
+ stateSinkObject.Save(0, &s.DynamicBytesFile)
+ stateSinkObject.Save(1, &s.StaticData)
+}
+
+func (s *staticControllerFile) afterLoad() {}
+
+// +checklocksignore
+func (s *staticControllerFile) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &s.DynamicBytesFile)
+ stateSourceObject.Load(1, &s.StaticData)
+}
+
+func (c *cpuController) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.cpuController"
+}
+
+func (c *cpuController) StateFields() []string {
+ return []string{
+ "controllerCommon",
+ "cfsPeriod",
+ "cfsQuota",
+ "shares",
+ }
+}
+
+func (c *cpuController) beforeSave() {}
+
+// +checklocksignore
+func (c *cpuController) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.controllerCommon)
+ stateSinkObject.Save(1, &c.cfsPeriod)
+ stateSinkObject.Save(2, &c.cfsQuota)
+ stateSinkObject.Save(3, &c.shares)
+}
+
+func (c *cpuController) afterLoad() {}
+
+// +checklocksignore
+func (c *cpuController) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.controllerCommon)
+ stateSourceObject.Load(1, &c.cfsPeriod)
+ stateSourceObject.Load(2, &c.cfsQuota)
+ stateSourceObject.Load(3, &c.shares)
+}
+
+func (c *cpuacctController) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.cpuacctController"
+}
+
+func (c *cpuacctController) StateFields() []string {
+ return []string{
+ "controllerCommon",
+ }
+}
+
+func (c *cpuacctController) beforeSave() {}
+
+// +checklocksignore
+func (c *cpuacctController) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.controllerCommon)
+}
+
+func (c *cpuacctController) afterLoad() {}
+
+// +checklocksignore
+func (c *cpuacctController) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.controllerCommon)
+}
+
+func (c *cpusetController) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.cpusetController"
+}
+
+func (c *cpusetController) StateFields() []string {
+ return []string{
+ "controllerCommon",
+ }
+}
+
+func (c *cpusetController) beforeSave() {}
+
+// +checklocksignore
+func (c *cpusetController) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.controllerCommon)
+}
+
+func (c *cpusetController) afterLoad() {}
+
+// +checklocksignore
+func (c *cpusetController) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.controllerCommon)
+}
+
+func (r *dirRefs) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.dirRefs"
+}
+
+func (r *dirRefs) StateFields() []string {
+ return []string{
+ "refCount",
+ }
+}
+
+func (r *dirRefs) beforeSave() {}
+
+// +checklocksignore
+func (r *dirRefs) StateSave(stateSinkObject state.Sink) {
+ r.beforeSave()
+ stateSinkObject.Save(0, &r.refCount)
+}
+
+// +checklocksignore
+func (r *dirRefs) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &r.refCount)
+ stateSourceObject.AfterLoad(r.afterLoad)
+}
+
+func (c *memoryController) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.memoryController"
+}
+
+func (c *memoryController) StateFields() []string {
+ return []string{
+ "controllerCommon",
+ }
+}
+
+func (c *memoryController) beforeSave() {}
+
+// +checklocksignore
+func (c *memoryController) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.controllerCommon)
+}
+
+func (c *memoryController) afterLoad() {}
+
+// +checklocksignore
+func (c *memoryController) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.controllerCommon)
+}
+
+func (d *memoryUsageInBytesData) StateTypeName() string {
+ return "pkg/sentry/fsimpl/cgroupfs.memoryUsageInBytesData"
+}
+
+func (d *memoryUsageInBytesData) StateFields() []string {
+ return []string{}
+}
+
+func (d *memoryUsageInBytesData) beforeSave() {}
+
+// +checklocksignore
+func (d *memoryUsageInBytesData) StateSave(stateSinkObject state.Sink) {
+ d.beforeSave()
+}
+
+func (d *memoryUsageInBytesData) afterLoad() {}
+
+// +checklocksignore
+func (d *memoryUsageInBytesData) StateLoad(stateSourceObject state.Source) {
+}
+
+func init() {
+ state.Register((*controllerCommon)(nil))
+ state.Register((*cgroupInode)(nil))
+ state.Register((*cgroupProcsData)(nil))
+ state.Register((*tasksData)(nil))
+ state.Register((*FilesystemType)(nil))
+ state.Register((*filesystem)(nil))
+ state.Register((*implStatFS)(nil))
+ state.Register((*dir)(nil))
+ state.Register((*controllerFile)(nil))
+ state.Register((*staticControllerFile)(nil))
+ state.Register((*cpuController)(nil))
+ state.Register((*cpuacctController)(nil))
+ state.Register((*cpusetController)(nil))
+ state.Register((*dirRefs)(nil))
+ state.Register((*memoryController)(nil))
+ state.Register((*memoryUsageInBytesData)(nil))
+}
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/dir_refs.go b/pkg/sentry/fsimpl/cgroupfs/dir_refs.go
new file mode 100644
index 000000000..741d0cddf
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/dir_refs.go
@@ -0,0 +1,132 @@
+package cgroupfs
+
+import (
+ "fmt"
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/refsvfs2"
+)
+
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const direnableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var dirobj *dir
+
+// Refs implements refs.RefCounter. It keeps a reference count using atomic
+// operations and calls the destructor when the count reaches zero.
+//
+// +stateify savable
+type dirRefs struct {
+ // refCount is composed of two fields:
+ //
+ // [32-bit speculative references]:[32-bit real references]
+ //
+ // Speculative references are used for TryIncRef, to avoid a CompareAndSwap
+ // loop. See IncRef, DecRef and TryIncRef for details of how these fields are
+ // used.
+ refCount int64
+}
+
+// InitRefs initializes r with one reference and, if enabled, activates leak
+// checking.
+func (r *dirRefs) InitRefs() {
+ atomic.StoreInt64(&r.refCount, 1)
+ refsvfs2.Register(r)
+}
+
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *dirRefs) RefType() string {
+ return fmt.Sprintf("%T", dirobj)[1:]
+}
+
+// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
+func (r *dirRefs) LeakMessage() string {
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *dirRefs) LogRefs() bool {
+ return direnableLogging
+}
+
+// ReadRefs returns the current number of references. The returned count is
+// inherently racy and is unsafe to use without external synchronization.
+func (r *dirRefs) ReadRefs() int64 {
+ return atomic.LoadInt64(&r.refCount)
+}
+
+// IncRef implements refs.RefCounter.IncRef.
+//
+//go:nosplit
+func (r *dirRefs) IncRef() {
+ v := atomic.AddInt64(&r.refCount, 1)
+ if direnableLogging {
+ refsvfs2.LogIncRef(r, v)
+ }
+ if v <= 1 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
+ }
+}
+
+// TryIncRef implements refs.RefCounter.TryIncRef.
+//
+// To do this safely without a loop, a speculative reference is first acquired
+// on the object. This allows multiple concurrent TryIncRef calls to distinguish
+// other TryIncRef calls from genuine references held.
+//
+//go:nosplit
+func (r *dirRefs) TryIncRef() bool {
+ const speculativeRef = 1 << 32
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) == 0 {
+
+ atomic.AddInt64(&r.refCount, -speculativeRef)
+ return false
+ }
+
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ if direnableLogging {
+ refsvfs2.LogTryIncRef(r, v)
+ }
+ return true
+}
+
+// DecRef implements refs.RefCounter.DecRef.
+//
+// Note that speculative references are counted here. Since they were added
+// prior to real references reaching zero, they will successfully convert to
+// real references. In other words, we see speculative references only in the
+// following case:
+//
+// A: TryIncRef [speculative increase => sees non-negative references]
+// B: DecRef [real decrease]
+// A: TryIncRef [transform speculative to real]
+//
+//go:nosplit
+func (r *dirRefs) DecRef(destroy func()) {
+ v := atomic.AddInt64(&r.refCount, -1)
+ if direnableLogging {
+ refsvfs2.LogDecRef(r, v)
+ }
+ switch {
+ case v < 0:
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
+
+ case v == 0:
+ refsvfs2.Unregister(r)
+
+ if destroy != nil {
+ destroy()
+ }
+ }
+}
+
+func (r *dirRefs) afterLoad() {
+ if r.ReadRefs() > 0 {
+ refsvfs2.Register(r)
+ }
+}
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/proc_state_autogen.go b/pkg/sentry/fsimpl/proc/proc_state_autogen.go
index 60e3a7485..8303e846d 100644
--- a/pkg/sentry/fsimpl/proc/proc_state_autogen.go
+++ b/pkg/sentry/fsimpl/proc/proc_state_autogen.go
@@ -1233,6 +1233,34 @@ func (fd *namespaceFD) StateLoad(stateSourceObject state.Source) {
stateSourceObject.Load(3, &fd.inode)
}
+func (d *taskCgroupData) StateTypeName() string {
+ return "pkg/sentry/fsimpl/proc.taskCgroupData"
+}
+
+func (d *taskCgroupData) StateFields() []string {
+ return []string{
+ "dynamicBytesFileSetAttr",
+ "task",
+ }
+}
+
+func (d *taskCgroupData) beforeSave() {}
+
+// +checklocksignore
+func (d *taskCgroupData) StateSave(stateSinkObject state.Sink) {
+ d.beforeSave()
+ stateSinkObject.Save(0, &d.dynamicBytesFileSetAttr)
+ stateSinkObject.Save(1, &d.task)
+}
+
+func (d *taskCgroupData) afterLoad() {}
+
+// +checklocksignore
+func (d *taskCgroupData) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &d.dynamicBytesFileSetAttr)
+ stateSourceObject.Load(1, &d.task)
+}
+
func (r *taskInodeRefs) StateTypeName() string {
return "pkg/sentry/fsimpl/proc.taskInodeRefs"
}
@@ -1554,7 +1582,7 @@ func (i *tasksInode) StateFields() []string {
"locks",
"fs",
"pidns",
- "cgroupControllers",
+ "fakeCgroupControllers",
}
}
@@ -1574,7 +1602,7 @@ func (i *tasksInode) StateSave(stateSinkObject state.Sink) {
stateSinkObject.Save(8, &i.locks)
stateSinkObject.Save(9, &i.fs)
stateSinkObject.Save(10, &i.pidns)
- stateSinkObject.Save(11, &i.cgroupControllers)
+ stateSinkObject.Save(11, &i.fakeCgroupControllers)
}
func (i *tasksInode) afterLoad() {}
@@ -1592,7 +1620,7 @@ func (i *tasksInode) StateLoad(stateSourceObject state.Source) {
stateSourceObject.Load(8, &i.locks)
stateSourceObject.Load(9, &i.fs)
stateSourceObject.Load(10, &i.pidns)
- stateSourceObject.Load(11, &i.cgroupControllers)
+ stateSourceObject.Load(11, &i.fakeCgroupControllers)
}
func (s *staticFileSetStat) StateTypeName() string {
@@ -1924,6 +1952,31 @@ func (d *filesystemsData) StateLoad(stateSourceObject state.Source) {
stateSourceObject.Load(0, &d.DynamicBytesFile)
}
+func (c *cgroupsData) StateTypeName() string {
+ return "pkg/sentry/fsimpl/proc.cgroupsData"
+}
+
+func (c *cgroupsData) StateFields() []string {
+ return []string{
+ "dynamicBytesFileSetAttr",
+ }
+}
+
+func (c *cgroupsData) beforeSave() {}
+
+// +checklocksignore
+func (c *cgroupsData) StateSave(stateSinkObject state.Sink) {
+ c.beforeSave()
+ stateSinkObject.Save(0, &c.dynamicBytesFileSetAttr)
+}
+
+func (c *cgroupsData) afterLoad() {}
+
+// +checklocksignore
+func (c *cgroupsData) StateLoad(stateSourceObject state.Source) {
+ stateSourceObject.Load(0, &c.dynamicBytesFileSetAttr)
+}
+
func (r *tasksInodeRefs) StateTypeName() string {
return "pkg/sentry/fsimpl/proc.tasksInodeRefs"
}
@@ -2231,6 +2284,7 @@ func init() {
state.Register((*namespaceSymlink)(nil))
state.Register((*namespaceInode)(nil))
state.Register((*namespaceFD)(nil))
+ state.Register((*taskCgroupData)(nil))
state.Register((*taskInodeRefs)(nil))
state.Register((*ifinet6)(nil))
state.Register((*netDevData)(nil))
@@ -2254,6 +2308,7 @@ func init() {
state.Register((*uptimeData)(nil))
state.Register((*versionData)(nil))
state.Register((*filesystemsData)(nil))
+ state.Register((*cgroupsData)(nil))
state.Register((*tasksInodeRefs)(nil))
state.Register((*tcpMemDir)(nil))
state.Register((*mmapMinAddrData)(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
+}