cgroupfs: Implement pids controller.

This also introduces the controller charge interface.

PiperOrigin-RevId: 444703063
This commit is contained in:
Rahat Mahmood 2022-04-26 16:47:47 -07:00 committed by gVisor bot
parent b5a59f96c8
commit bf251f1838
14 changed files with 616 additions and 40 deletions

View File

@ -26,6 +26,7 @@ go_library(
"dir_refs.go",
"job.go",
"memory.go",
"pids.go",
],
visibility = ["//pkg/sentry:internal"],
deps = [

View File

@ -138,6 +138,10 @@ type controller interface {
//
// Precondition: Caller must call a corresponding PrepareMigrate.
AbortMigrate(t *kernel.Task, src controller)
// Charge charges a controller for a particular resource. The implementation
// should panic if passed a resource type they do not control.
Charge(t *kernel.Task, d *kernfs.Dentry, res kernel.CgroupResourceType, value int64) error
}
// cgroupInode implements kernel.CgroupImpl and kernfs.Inode.
@ -182,10 +186,11 @@ func (fs *filesystem) newCgroupInode(ctx context.Context, creds *auth.Credential
}
} else {
for _, ctl := range fs.controllers {
new := ctl.Clone()
// Uniqueness of controllers enforced by the filesystem on creation.
c.controllers[ctl.Type()] = new
new.AddControlFiles(ctx, creds, c, contents)
// Uniqueness of controllers enforced by the filesystem on
// creation. The root cgroup uses the controllers directly from the
// filesystem.
c.controllers[ctl.Type()] = ctl
ctl.AddControlFiles(ctx, creds, c, contents)
}
}
@ -282,6 +287,7 @@ func (c *cgroupInode) AbortMigrate(t *kernel.Task, src *kernel.Cgroup) {
}
}
// CgroupFromControlFileFD returns a cgroup object given a control file FD for the cgroup.
func (c *cgroupInode) CgroupFromControlFileFD(fd *vfs.FileDescription) kernel.Cgroup {
controlFileDentry := fd.Dentry().Impl().(*kernfs.Dentry)
// The returned parent dentry remains valid without holding locks because in
@ -295,6 +301,22 @@ func (c *cgroupInode) CgroupFromControlFileFD(fd *vfs.FileDescription) kernel.Cg
}
}
// Charge implements kernel.CgroupImpl.Charge.
//
// Charge notifies a matching controller of a change in resource usage. Due to
// the uniqueness of controllers, at most one controller will match. If no
// matching controller is present in this directory, the call silently
// succeeds. The caller should call Charge on all hierarchies to ensure any
// matching controller across the entire system is charged.
func (c *cgroupInode) Charge(t *kernel.Task, d *kernfs.Dentry, ctlType kernel.CgroupControllerType, res kernel.CgroupResourceType, value int64) error {
c.fs.tasksMu.RLock()
defer c.fs.tasksMu.RUnlock()
if ctl, ok := c.controllers[ctlType]; ok {
return ctl.Charge(t, d, res, value)
}
return nil
}
func sortTIDs(tids []kernel.ThreadID) {
sort.Slice(tids, func(i, j int) bool { return tids[i] < tids[j] })
}
@ -435,3 +457,13 @@ func (*controllerStateless) CommitMigrate(t *kernel.Task, src controller) {}
// AbortMigrate implements controller.AbortMigrate.
func (*controllerStateless) AbortMigrate(t *kernel.Task, src controller) {}
// controllerNoResource partially implements controller. It stubs out the Charge
// method for controllers that don't track resource usage through the charge
// mechanism.
type controllerNoResource struct{}
// Charge implements controller.Charge.
func (*controllerNoResource) Charge(t *kernel.Task, d *kernfs.Dentry, res kernel.CgroupResourceType, value int64) error {
panic(fmt.Sprintf("cgroupfs: Attempted to charge a controller with unknown resource %v for value %v", res, value))
}

View File

@ -86,24 +86,17 @@ const (
defaultMaxCachedDentries = uint64(1000)
)
const (
controllerCPU = kernel.CgroupControllerType("cpu")
controllerCPUAcct = kernel.CgroupControllerType("cpuacct")
controllerCPUSet = kernel.CgroupControllerType("cpuset")
controllerJob = kernel.CgroupControllerType("job")
controllerMemory = kernel.CgroupControllerType("memory")
)
var allControllers = []kernel.CgroupControllerType{
controllerCPU,
controllerCPUAcct,
controllerCPUSet,
controllerJob,
controllerMemory,
kernel.CgroupControllerCPU,
kernel.CgroupControllerCPUAcct,
kernel.CgroupControllerCPUSet,
kernel.CgroupControllerJob,
kernel.CgroupControllerMemory,
kernel.CgroupControllerPIDs,
}
// SupportedMountOptions is the set of supported mount options for cgroupfs.
var SupportedMountOptions = []string{"all", "cpu", "cpuacct", "cpuset", "job", "memory"}
var SupportedMountOptions = []string{"all", "cpu", "cpuacct", "cpuset", "job", "memory", "pids"}
// FilesystemType implements vfs.FilesystemType.
//
@ -188,23 +181,27 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
var wantControllers []kernel.CgroupControllerType
if _, ok := mopts["cpu"]; ok {
delete(mopts, "cpu")
wantControllers = append(wantControllers, controllerCPU)
wantControllers = append(wantControllers, kernel.CgroupControllerCPU)
}
if _, ok := mopts["cpuacct"]; ok {
delete(mopts, "cpuacct")
wantControllers = append(wantControllers, controllerCPUAcct)
wantControllers = append(wantControllers, kernel.CgroupControllerCPUAcct)
}
if _, ok := mopts["cpuset"]; ok {
delete(mopts, "cpuset")
wantControllers = append(wantControllers, controllerCPUSet)
wantControllers = append(wantControllers, kernel.CgroupControllerCPUSet)
}
if _, ok := mopts["job"]; ok {
delete(mopts, "job")
wantControllers = append(wantControllers, controllerJob)
wantControllers = append(wantControllers, kernel.CgroupControllerJob)
}
if _, ok := mopts["memory"]; ok {
delete(mopts, "memory")
wantControllers = append(wantControllers, controllerMemory)
wantControllers = append(wantControllers, kernel.CgroupControllerMemory)
}
if _, ok := mopts["pids"]; ok {
delete(mopts, "pids")
wantControllers = append(wantControllers, kernel.CgroupControllerPIDs)
}
if _, ok := mopts["all"]; ok {
if len(wantControllers) > 0 {
@ -269,16 +266,18 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
for _, ty := range wantControllers {
var c controller
switch ty {
case controllerCPU:
case kernel.CgroupControllerCPU:
c = newCPUController(fs, defaults)
case controllerCPUAcct:
case kernel.CgroupControllerCPUAcct:
c = newCPUAcctController(fs)
case controllerCPUSet:
case kernel.CgroupControllerCPUSet:
c = newCPUSetController(k, fs)
case controllerJob:
case kernel.CgroupControllerJob:
c = newJobController(fs)
case controllerMemory:
case kernel.CgroupControllerMemory:
c = newMemoryController(fs, defaults)
case kernel.CgroupControllerPIDs:
c = newRootPIDsController(fs)
default:
panic(fmt.Sprintf("Unreachable: unknown cgroup controller %q", ty))
}

View File

@ -18,6 +18,7 @@ import (
"gvisor.dev/gvisor/pkg/atomicbitops"
"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"
)
@ -25,6 +26,7 @@ import (
type cpuController struct {
controllerCommon
controllerStateless
controllerNoResource
// CFS bandwidth control parameters, values in microseconds.
cfsPeriod atomicbitops.Int64
@ -57,7 +59,7 @@ func newCPUController(fs *filesystem, defaults map[string]int64) *cpuController
delete(defaults, "cpu.shares")
}
c.controllerCommon.init(controllerCPU, fs)
c.controllerCommon.init(kernel.CgroupControllerCPU, fs)
return c
}

View File

@ -44,7 +44,7 @@ import (
// +stateify savable
type cpuacctController struct {
controllerCommon
controllerStateless
controllerNoResource
mu sync.Mutex `state:"nosave"`
@ -65,7 +65,7 @@ func newCPUAcctController(fs *filesystem) *cpuacctController {
c := &cpuacctController{
taskCommittedCharges: make(map[*kernel.Task]usage.CPUStats),
}
c.controllerCommon.init(controllerCPUAcct, fs)
c.controllerCommon.init(kernel.CgroupControllerCPUAcct, fs)
return c
}
@ -133,7 +133,7 @@ type cpuacctCgroup struct {
}
func (c *cpuacctCgroup) cpuacctController() *cpuacctController {
return c.controllers[controllerCPUAcct].(*cpuacctController)
return c.controllers[kernel.CgroupControllerCPUAcct].(*cpuacctController)
}
// checklocks:c.fs.tasksMu

View File

@ -35,6 +35,7 @@ import (
type cpusetController struct {
controllerCommon
controllerStateless
controllerNoResource
maxCpus uint32
maxMems uint32
@ -59,7 +60,7 @@ func newCPUSetController(k *kernel.Kernel, fs *filesystem) *cpusetController {
maxCpus: uint32(k.ApplicationCores()),
maxMems: 1, // We always report a single NUMA node.
}
c.controllerCommon.init(controllerCPUSet, fs)
c.controllerCommon.init(kernel.CgroupControllerCPUSet, fs)
return c
}

View File

@ -18,6 +18,7 @@ import (
"gvisor.dev/gvisor/pkg/atomicbitops"
"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"
)
@ -25,6 +26,7 @@ import (
type jobController struct {
controllerCommon
controllerStateless
controllerNoResource
id atomicbitops.Int64
}
@ -33,7 +35,7 @@ var _ controller = (*jobController)(nil)
func newJobController(fs *filesystem) *jobController {
c := &jobController{}
c.controllerCommon.init(controllerJob, fs)
c.controllerCommon.init(kernel.CgroupControllerJob, fs)
return c
}

View File

@ -32,6 +32,7 @@ import (
type memoryController struct {
controllerCommon
controllerStateless
controllerNoResource
limitBytes atomicbitops.Int64
softLimitBytes atomicbitops.Int64
@ -62,7 +63,7 @@ func newMemoryController(fs *filesystem, defaults map[string]int64) *memoryContr
consumeDefault("memory.soft_limit_in_bytes", &c.softLimitBytes)
consumeDefault("memory.move_charge_at_immigrate", &c.moveChargeAtImmigrate)
c.controllerCommon.init(controllerMemory, fs)
c.controllerCommon.init(kernel.CgroupControllerMemory, fs)
return c
}

View File

@ -0,0 +1,296 @@
// Copyright 2022 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"
"strings"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"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/usermem"
)
// pidMaxLimit is the maximum number of pids allowed on a 64-bit system. The
// practical limit is much lower. See Linux, include/linux/threads.h.
const pidMaxLimit = 4 * 1024 * 1024
const pidLimitUnlimited = pidMaxLimit + 1
// pidsController tracks how many pids are used by tasks in a cgroup. This is
// used to limit the number of tasks per cgroup. The limit is enforced only when
// new tasks are created via Fork/Clone. Task migrations and limit changes can
// cause the current number of pids to exceed the limit.
//
// A task can charge a PIDs cgroup in two ways:
//
// 1) A task created prior to the PIDs controller being enabled, or created
// through kernel.CreateProcess (i.e. not from userspace) directly add
// committed charges via the Enter method.
//
// 2) A task created through Task.Clone (i.e. userspace fork/clone) first add a
// pending charge through the Charge method. This is a temporary reservation
// which ensures the cgroup has enough space to allow the task to start. Once
// the task startup succeeds, it calls Enter and consumes the reservation.
//
// +stateify savable
type pidsController struct {
controllerCommon
// isRoot indiciates if this is the root cgroup in its hierarchy. Immutable
// since cgroupfs doesn't allow cross directory renames.
isRoot bool
// mu protects the fields below.
mu sync.Mutex `state:"nosave"`
// pendingTotal and pendingPool tracks the charge for processes starting
// up. During startup, we check if PIDs are available by charging the
// cgroup. However, the process actually joins the cgroup as a later point
// via Enter. We keep a count of the charges we allocated via Charge, and
// use this pool to account for already accounted charges from Enter.
//
// We also track which task owns the pending charge so we can cancel the
// charge if a task creation fails after the Charge call.
//
// pendingTotal and pendingPool are both protected by mu.
pendingTotal int64
pendingPool map[*kernel.Task]int64
// committed represent charges for tasks that have already started and
// called Enter. Protected by mu.
committed int64
// max is the PID limit for this cgroup. Protected by mu.
max int64
}
var _ controller = (*pidsController)(nil)
// newRootPIDsController creates the root node for a PIDs cgroup. Child
// directories should be created through Clone.
func newRootPIDsController(fs *filesystem) *pidsController {
c := &pidsController{
isRoot: true,
max: pidLimitUnlimited,
pendingPool: make(map[*kernel.Task]int64),
}
c.controllerCommon.init(kernel.CgroupControllerPIDs, fs)
return c
}
// Clone implements controller.Clone.
func (c *pidsController) Clone() controller {
c.mu.Lock()
defer c.mu.Unlock()
new := &pidsController{
isRoot: false,
max: pidLimitUnlimited,
pendingPool: make(map[*kernel.Task]int64),
}
new.controllerCommon.cloneFromParent(c)
return new
}
// AddControlFiles implements controller.AddControlFiles.
func (c *pidsController) AddControlFiles(ctx context.Context, creds *auth.Credentials, _ *cgroupInode, contents map[string]kernfs.Inode) {
contents["pids.current"] = c.fs.newControllerFile(ctx, creds, &pidsCurrentData{c: c})
if !c.isRoot {
// "This is not available in the root cgroup for obvious reasons" --
// Linux, Documentation/cgroup-v1/pids.txt.
contents["pids.max"] = c.fs.newControllerWritableFile(ctx, creds, &pidsMaxData{c: c})
}
}
// Enter implements controller.Enter.
//
// Enter attempts to commit a charge from the pending pool. If at least one
// charge is pending for t, one pending charge is converted to a commited
// charge, and the net change in total charges is zero. If no charge is pending,
// a new charge is added directly to the committed pool.
func (c *pidsController) Enter(t *kernel.Task) {
c.mu.Lock()
defer c.mu.Unlock()
if pending, ok := c.pendingPool[t]; ok {
if pending == 1 {
delete(c.pendingPool, t)
} else {
c.pendingPool[t] = pending - 1
}
c.pendingTotal--
if c.pendingTotal < 0 {
panic(fmt.Sprintf("cgroupfs: pids controller has negative pending charge: %v\n", c.committed))
}
}
// Either we're converting a pending charge from above, or generating a new
// committed charge directly here. Either way, we don't enforce the limit on
// Enter.
c.committed++
}
// Leave implements controller.Leave.
func (c *pidsController) Leave(t *kernel.Task) {
c.mu.Lock()
defer c.mu.Unlock()
if c.committed <= 0 {
panic(fmt.Sprintf("cgroupfs: pids controller committed charge underflow on Leave for task %+v", t))
}
c.committed--
}
// PrepareMigrate implements controller.PrepareMigrate.
func (c *pidsController) PrepareMigrate(t *kernel.Task, src controller) error {
return nil
}
// CommitMigrate implements controller.CommitMigrate.
//
// Migrations can cause a cgroup to exceed its limit. Migration can only be
// called for tasks with committed charges, as it is not possible to migrate a
// task prior to Enter.
func (c *pidsController) CommitMigrate(t *kernel.Task, src controller) {
// Note: The charge is allowed to exceed max on migration. The charge may
// not exceed max when incurred due to a fork/clone, which will call
// pidsController.Charge().
c.mu.Lock()
c.committed++
c.mu.Unlock()
srcC := src.(*pidsController)
srcC.mu.Lock()
if srcC.committed <= 0 {
panic(fmt.Sprintf("cgroupfs: pids controller committed charge underflow on CommitMigrate for task %+v on the source cgroup", t))
}
srcC.committed--
srcC.mu.Unlock()
}
// AbortMigrate implements controller.AbortMigrate.
func (c *pidsController) AbortMigrate(t *kernel.Task, src controller) {}
// Charge implements controller.Charge. This manipulates the pending
// pool. Charge are committed from the pending pool by Enter. The caller is
// responsible for ensuring negative charges correspond to previous positive
// charges. Negative charges that cause an underflow result in a panic.
func (c *pidsController) Charge(t *kernel.Task, d *kernfs.Dentry, res kernel.CgroupResourceType, value int64) error {
if res != kernel.CgroupResourcePID {
panic(fmt.Sprintf("cgroupfs: pids controller invalid resource type %v", res))
}
c.mu.Lock()
defer c.mu.Unlock()
// Negative charge.
if value < 0 {
if c.pendingTotal+value < 0 {
panic(fmt.Sprintf("cgroupfs: pids controller pending pool would be negative if charge was allowed: current pool: %d, proposed charge: %d", c.pendingTotal, value))
}
pending, ok := c.pendingPool[t]
if !ok {
panic(fmt.Sprintf("cgroupfs: pids controller attempted to remove pending charge for task %+v, but task didn't have pending charges", t))
}
if pending+value < 0 {
panic(fmt.Sprintf("cgroupfs: pids controller attempted to remove pending charge for task %+v, but task didn't have enough pending charges; current charges: %d, proposed charge: %d", t, pending, value))
}
c.pendingPool[t] += value
c.pendingTotal += value
return nil
}
// Positive charge.
new := c.committed + c.pendingTotal + value
if new > c.max {
log.Debugf("cgroupfs: pids controller charge denied due to limit: path: %q, requested: %d, current: %d (pending: %v, committed: %v), max: %v",
d.FSLocalPath(), value, c.committed+c.pendingTotal, c.pendingTotal, c.committed, c.max)
return linuxerr.EAGAIN
}
c.pendingPool[t] += value
c.pendingTotal += value
return nil
}
// +stateify savable
type pidsCurrentData struct {
c *pidsController
}
// Generate implements vfs.DynamicBytesSource.Generate.
func (d *pidsCurrentData) Generate(ctx context.Context, buf *bytes.Buffer) error {
d.c.mu.Lock()
defer d.c.mu.Unlock()
fmt.Fprintf(buf, "%d\n", d.c.committed+d.c.pendingTotal)
return nil
}
// +stateify savable
type pidsMaxData struct {
c *pidsController
}
// Generate implements vfs.DynamicBytesSource.Generate.
func (d *pidsMaxData) Generate(ctx context.Context, buf *bytes.Buffer) error {
d.c.mu.Lock()
defer d.c.mu.Unlock()
if d.c.max > pidMaxLimit {
fmt.Fprintf(buf, "max\n")
} else {
fmt.Fprintf(buf, "%d\n", d.c.max)
}
return nil
}
// Write implements vfs.WritableDynamicBytesSource.Write.
func (d *pidsMaxData) Write(ctx context.Context, _ *vfs.FileDescription, src usermem.IOSequence, offset int64) (int64, error) {
d.c.mu.Lock()
defer d.c.mu.Unlock()
t := kernel.TaskFromContext(ctx)
buf := t.CopyScratchBuffer(hostarch.PageSize)
ncpy, err := src.CopyIn(ctx, buf)
if err != nil {
return 0, err
}
if strings.TrimSpace(string(buf)) == "max" {
d.c.max = pidLimitUnlimited
return int64(ncpy), nil
}
val, n, err := parseInt64FromString(ctx, src)
if err != nil {
return 0, linuxerr.EINVAL
}
if val < 0 || val > pidMaxLimit {
return 0, linuxerr.EINVAL
}
d.c.max = val
return int64(n), nil
}

View File

@ -32,6 +32,26 @@ const InvalidCgroupHierarchyID uint32 = 0
// CgroupControllerType is the name of a cgroup controller.
type CgroupControllerType string
// Available cgroup controllers.
const (
CgroupControllerCPU = CgroupControllerType("cpu")
CgroupControllerCPUAcct = CgroupControllerType("cpuacct")
CgroupControllerCPUSet = CgroupControllerType("cpuset")
CgroupControllerJob = CgroupControllerType("job")
CgroupControllerMemory = CgroupControllerType("memory")
CgroupControllerPIDs = CgroupControllerType("pids")
)
// CgroupResourceType represents a resource type tracked by a particular
// controller.
type CgroupResourceType int
// Resources for the cpuacct controller.
const (
// CgroupResourcePID represents a charge for pids.current.
CgroupResourcePID CgroupResourceType = iota
)
// CgroupController is the common interface to cgroup controllers available to
// the entire sentry. The controllers themselves are defined by cgroupfs.
//
@ -133,6 +153,18 @@ type CgroupImpl interface {
// AbortMigrate cancels an in-flight migration. See
// cgroupfs.controller.AbortMigrate.
AbortMigrate(t *Task, src *Cgroup)
// Charge charges a controller in this cgroup for a particular resource. key
// must match a valid resource for the specified controller type.
//
// The implementer should silently succeed if no matching controllers are
// found.
//
// The underlying implementaion will panic if passed an incompatible
// resource type for a given controller.
//
// See cgroupfs.controller.Charge.
Charge(t *Task, d *kernfs.Dentry, ctl CgroupControllerType, res CgroupResourceType, value int64) error
}
// hierarchy represents a cgroupfs filesystem instance, with a unique set of

View File

@ -26,7 +26,9 @@ import (
// EnterInitialCgroups moves t into an initial set of cgroups.
//
// Precondition: t isn't in any cgroups yet, t.cgs is empty.
// This is analogous to Linux's kernel/cgroup/cgroup.c:cgroup_css_set_fork().
//
// Precondition: t isn't in any cgroups yet, t.cgroups is empty.
func (t *Task) EnterInitialCgroups(parent *Task) {
var inherit map[Cgroup]struct{}
if parent != nil {
@ -212,3 +214,20 @@ func (t *Task) GenerateProcTaskCgroup(buf *bytes.Buffer) {
fmt.Fprintf(buf, "%d:%s:%s\n", cgE.hierarchyID, cgE.controllers, cgE.path)
}
}
// +checklocks:t.mu
func (t *Task) chargeLocked(target *Task, ctl CgroupControllerType, res CgroupResourceType, value int64) error {
for c, _ := range t.cgroups {
if err := c.Charge(target, c.Dentry, ctl, res, value); err != nil {
return err
}
}
return nil
}
// ChargeFor charges t's cgroup on behalf of some other task.
func (t *Task) ChargeFor(other *Task, ctl CgroupControllerType, res CgroupResourceType, value int64) error {
t.mu.Lock()
defer t.mu.Unlock()
return t.chargeLocked(other, ctl, res, value)
}

View File

@ -15,8 +15,11 @@
package kernel
import (
"fmt"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/atomicbitops"
"gvisor.dev/gvisor/pkg/cleanup"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
@ -120,7 +123,7 @@ func (ts *TaskSet) NewTask(ctx context.Context, cfg *TaskConfig) (*Task, error)
cleanup()
return nil, err
}
t, err := ts.newTask(cfg)
t, err := ts.newTask(ctx, cfg)
if err != nil {
cfg.UserCounters.decRLimitNProc()
cleanup()
@ -131,7 +134,8 @@ func (ts *TaskSet) NewTask(ctx context.Context, cfg *TaskConfig) (*Task, error)
// newTask is a helper for TaskSet.NewTask that only takes ownership of parts
// of cfg if it succeeds.
func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
func (ts *TaskSet) newTask(ctx context.Context, cfg *TaskConfig) (*Task, error) {
srcT := TaskFromContext(ctx)
tg := cfg.ThreadGroup
image := cfg.TaskImage
t := &Task{
@ -171,6 +175,27 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
// We don't construct t.blockingTimer until Task.run(); see that function
// for justification.
var cu cleanup.Cleanup
defer cu.Clean()
// Reserve cgroup PIDs controller charge. This is either commited when the
// new task enters the cgroup below, or rolled back on failure.
//
// We may also get here from a non-task context (for example, when
// creating the init task, or from the exec control command). In these cases
// we skip charging the pids controller, as non-userspace task creation
// bypasses pid limits.
if srcT != nil {
if err := srcT.ChargeFor(t, CgroupControllerPIDs, CgroupResourcePID, 1); err != nil {
return nil, err
}
cu.Add(func() {
if err := srcT.ChargeFor(t, CgroupControllerPIDs, CgroupResourcePID, -1); err != nil {
panic(fmt.Sprintf("Failed to clean up PIDs charge on task creation failure: %v", err))
}
})
}
// Make the new task (and possibly thread group) visible to the rest of
// the system atomically.
ts.mu.Lock()
@ -203,7 +228,8 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
}
if VFS2Enabled {
t.EnterInitialCgroups(t.parent)
// srcT may be nil, in which case we default to root cgroups.
t.EnterInitialCgroups(srcT)
}
if tg.leader == nil {
@ -237,6 +263,7 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
// other pieces to be initialized as the task is used the context.
t.p = cfg.Kernel.Platform.NewContext(t.AsyncContext())
cu.Release()
return t, nil
}

View File

@ -494,6 +494,14 @@ func (t *Task) Parent() *Task {
return t.parent
}
// ParentLocked returns t's parent. Caller must ensure t's TaskSet mu
// is locked for at least reading.
//
// +checklocks:t.tg.pidns.owner.mu
func (t *Task) ParentLocked() *Task {
return t.parent
}
// ThreadID returns t's thread ID in its own PID namespace. If the task is
// dead, ThreadID returns 0.
func (t *Task) ThreadID() ThreadID {

View File

@ -46,7 +46,7 @@ using ::testing::Key;
using ::testing::Not;
std::vector<std::string> known_controllers = {
"cpu", "cpuset", "cpuacct", "job", "memory",
"cpu", "cpuset", "cpuacct", "job", "memory", "pids",
};
bool CgroupsAvailable() {
@ -87,6 +87,8 @@ class NoopThreads {
bool joined_ = false;
};
void* DummyThreadBody(void* unused) { return nullptr; }
TEST(Cgroup, MountSucceeds) {
SKIP_IF(!CgroupsAvailable());
@ -1072,6 +1074,160 @@ TEST(ProcCgroups, ProcfsReportsTasksCgroup) {
EXPECT_EQ(h2c2.CanonicalPath(), entries["cpu,cpuacct"].path);
}
TEST(PIDsCgroup, ControlFilesExist) {
SKIP_IF(!CgroupsAvailable());
Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()));
Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("pids"));
// The pids.max file isn't available for the root cgroup.
EXPECT_THAT(c.ReadControlFile("pids.max"), PosixErrorIs(ENOENT, _));
// There should be at least one PID in use in the root controller, since the
// test process is running in the root controller.
const int64_t current =
ASSERT_NO_ERRNO_AND_VALUE(c.ReadIntegerControlFile("pids.current"));
EXPECT_GE(current, 1);
Cgroup child = ASSERT_NO_ERRNO_AND_VALUE(c.CreateChild("child"));
// The limit file should exist for any child cgroups, and should be unlimited
// by default.
const std::string child_limit =
ASSERT_NO_ERRNO_AND_VALUE(child.ReadControlFile("pids.max"));
EXPECT_EQ(child_limit, "max\n");
// The child cgroup should have no tasks, and thus no pids usage.
const int64_t current_child =
ASSERT_NO_ERRNO_AND_VALUE(child.ReadIntegerControlFile("pids.current"));
EXPECT_EQ(current_child, 0);
}
TEST(PIDsCgroup, ChargeMigration) {
SKIP_IF(!CgroupsAvailable());
Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()));
Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("pids"));
const int64_t root_start =
ASSERT_NO_ERRNO_AND_VALUE(c.ReadIntegerControlFile("pids.current"));
// Root should have at least one task.
ASSERT_GE(root_start, 1);
Cgroup child = ASSERT_NO_ERRNO_AND_VALUE(c.CreateChild("child"));
// Child initially has no charge.
EXPECT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(0));
// Move the test process. The root cgroup should lose charges equal to the
// number of tasks moved to the child.
ASSERT_NO_ERRNO(child.Enter(getpid()));
const int64_t child_after =
ASSERT_NO_ERRNO_AND_VALUE(child.ReadIntegerControlFile("pids.current"));
EXPECT_GE(child_after, 1);
const int64_t root_after =
ASSERT_NO_ERRNO_AND_VALUE(c.ReadIntegerControlFile("pids.current"));
EXPECT_EQ(root_start - root_after, child_after);
}
TEST(PIDsCgroup, MigrationCanExceedLimit) {
SKIP_IF(!CgroupsAvailable());
Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()));
Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("pids"));
Cgroup child = ASSERT_NO_ERRNO_AND_VALUE(c.CreateChild("child"));
// Set child limit to 0, and try move tasks into it. This should be allowed,
// as the limit isn't enforced on migration.
ASSERT_NO_ERRNO(child.WriteIntegerControlFile("pids.max", 0));
ASSERT_NO_ERRNO(child.Enter(getpid()));
EXPECT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(Gt(0)));
}
TEST(PIDsCgroup, SetInvalidLimit) {
SKIP_IF(!CgroupsAvailable());
Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()));
Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("pids"));
Cgroup child = ASSERT_NO_ERRNO_AND_VALUE(c.CreateChild("child"));
// Set a valid limit, so we can verify it doesn't change after invalid writes.
ASSERT_NO_ERRNO(child.WriteIntegerControlFile("pids.max", 1234));
EXPECT_THAT(child.WriteControlFile("pids.max", "m a x"),
PosixErrorIs(EINVAL, _));
EXPECT_THAT(child.WriteControlFile("pids.max", "some-invalid-string"),
PosixErrorIs(EINVAL, _));
EXPECT_THAT(child.WriteControlFile("pids.max", "-1"),
PosixErrorIs(EINVAL, _));
EXPECT_THAT(child.WriteControlFile("pids.max", "-3894732"),
PosixErrorIs(EINVAL, _));
// This value is much larger than the maximum allowed value of ~ 1<<22.
EXPECT_THAT(child.WriteIntegerControlFile("pids.max", LLONG_MAX - 1),
PosixErrorIs(EINVAL, _));
// The initial valid limit should remain unchanged.
EXPECT_THAT(child.ReadIntegerControlFile("pids.max"),
IsPosixErrorOkAndHolds(1234));
}
TEST(PIDsCgroup, CanLowerLimitBelowCurrentCharge) {
SKIP_IF(!CgroupsAvailable());
Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()));
Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("pids"));
Cgroup child = ASSERT_NO_ERRNO_AND_VALUE(c.CreateChild("child"));
ASSERT_NO_ERRNO(child.Enter(getpid()));
// Confirm current charge is non-zero.
ASSERT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(Gt(0)));
// Try set limit to zero.
EXPECT_NO_ERRNO(child.WriteIntegerControlFile("pids.max", 0));
}
TEST(PIDsCgroup, LimitEnforced) {
SKIP_IF(!CgroupsAvailable());
Mounter m(ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()));
Cgroup c = ASSERT_NO_ERRNO_AND_VALUE(m.MountCgroupfs("pids"));
Cgroup child = ASSERT_NO_ERRNO_AND_VALUE(c.CreateChild("child"));
ASSERT_NO_ERRNO(child.Enter(getpid()));
// Set limit to so we have room for one more task.
const int64_t baseline =
ASSERT_NO_ERRNO_AND_VALUE(child.ReadIntegerControlFile("pids.current"));
ASSERT_NO_ERRNO(child.WriteIntegerControlFile("pids.max", baseline + 1));
// Spawning a thread should succeed.
NoopThreads t1(1);
ASSERT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(baseline + 1));
// Attempting to spawn another thread should fail.
pthread_t pt;
EXPECT_EQ(pthread_create(&pt, nullptr, &DummyThreadBody, nullptr), EAGAIN);
ASSERT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(baseline + 1));
// Exit the first thread and try create a thread again, which should succeed.
t1.Join();
ASSERT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(baseline));
NoopThreads t2(1);
EXPECT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(baseline + 1));
// Increase the limit and try again.
ASSERT_NO_ERRNO(child.WriteIntegerControlFile("pids.max", baseline + 2));
NoopThreads t3(1);
EXPECT_THAT(child.ReadIntegerControlFile("pids.current"),
IsPosixErrorOkAndHolds(baseline + 2));
}
} // namespace
} // namespace testing
} // namespace gvisor