diff options
author | Rahat Mahmood <rahat@google.com> | 2021-04-02 21:08:53 -0700 |
---|---|---|
committer | gVisor bot <gvisor-bot@google.com> | 2021-04-02 21:10:44 -0700 |
commit | 932c8abd0f739bec295ff62cf8fce3dcb7e2d866 (patch) | |
tree | 5669f3240478e7d96a9d9375d241e65668cb5805 /pkg/sentry/kernel | |
parent | a0c1674478ed49b63c75ce1d000c8038b1a632b3 (diff) |
Implement cgroupfs.
A skeleton implementation of cgroupfs. It supports trivial cpu and
memory controllers with no support for hierarchies.
PiperOrigin-RevId: 366561126
Diffstat (limited to 'pkg/sentry/kernel')
-rw-r--r-- | pkg/sentry/kernel/BUILD | 3 | ||||
-rw-r--r-- | pkg/sentry/kernel/cgroup.go | 281 | ||||
-rw-r--r-- | pkg/sentry/kernel/kernel.go | 52 | ||||
-rw-r--r-- | pkg/sentry/kernel/task.go | 6 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_cgroup.go | 138 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_exit.go | 4 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_start.go | 5 | ||||
-rw-r--r-- | pkg/sentry/kernel/threads.go | 9 |
8 files changed, 498 insertions, 0 deletions
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. // |