open-nomad/drivers/shared/executor/executor_linux.go

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

927 lines
27 KiB
Go
Raw Normal View History

//go:build linux
2016-02-05 00:03:17 +00:00
package executor
2016-02-02 21:38:38 +00:00
import (
"context"
client: enable support for cgroups v2 This PR introduces support for using Nomad on systems with cgroups v2 [1] enabled as the cgroups controller mounted on /sys/fs/cgroups. Newer Linux distros like Ubuntu 21.10 are shipping with cgroups v2 only, causing problems for Nomad users. Nomad mostly "just works" with cgroups v2 due to the indirection via libcontainer, but not so for managing cpuset cgroups. Before, Nomad has been making use of a feature in v1 where a PID could be a member of more than one cgroup. In v2 this is no longer possible, and so the logic around computing cpuset values must be modified. When Nomad detects v2, it manages cpuset values in-process, rather than making use of cgroup heirarchy inheritence via shared/reserved parents. Nomad will only activate the v2 logic when it detects cgroups2 is mounted at /sys/fs/cgroups. This means on systems running in hybrid mode with cgroups2 mounted at /sys/fs/cgroups/unified (as is typical) Nomad will continue to use the v1 logic, and should operate as before. Systems that do not support cgroups v2 are also not affected. When v2 is activated, Nomad will create a parent called nomad.slice (unless otherwise configured in Client conifg), and create cgroups for tasks using naming convention <allocID>-<task>.scope. These follow the naming convention set by systemd and also used by Docker when cgroups v2 is detected. Client nodes now export a new fingerprint attribute, unique.cgroups.version which will be set to 'v1' or 'v2' to indicate the cgroups regime in use by Nomad. The new cpuset management strategy fixes #11705, where docker tasks that spawned processes on startup would "leak". In cgroups v2, the PIDs are started in the cgroup they will always live in, and thus the cause of the leak is eliminated. [1] https://www.kernel.org/doc/html/latest/admin-guide/cgroup-v2.html Closes #11289 Fixes #11705 #11773 #11933
2022-02-28 22:24:01 +00:00
"errors"
"fmt"
"io"
"os"
"os/exec"
"path"
"path/filepath"
"strings"
"syscall"
2016-05-09 14:57:26 +00:00
"time"
"github.com/armon/circbuf"
"github.com/hashicorp/consul-template/signals"
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
client: enable support for cgroups v2 This PR introduces support for using Nomad on systems with cgroups v2 [1] enabled as the cgroups controller mounted on /sys/fs/cgroups. Newer Linux distros like Ubuntu 21.10 are shipping with cgroups v2 only, causing problems for Nomad users. Nomad mostly "just works" with cgroups v2 due to the indirection via libcontainer, but not so for managing cpuset cgroups. Before, Nomad has been making use of a feature in v1 where a PID could be a member of more than one cgroup. In v2 this is no longer possible, and so the logic around computing cpuset values must be modified. When Nomad detects v2, it manages cpuset values in-process, rather than making use of cgroup heirarchy inheritence via shared/reserved parents. Nomad will only activate the v2 logic when it detects cgroups2 is mounted at /sys/fs/cgroups. This means on systems running in hybrid mode with cgroups2 mounted at /sys/fs/cgroups/unified (as is typical) Nomad will continue to use the v1 logic, and should operate as before. Systems that do not support cgroups v2 are also not affected. When v2 is activated, Nomad will create a parent called nomad.slice (unless otherwise configured in Client conifg), and create cgroups for tasks using naming convention <allocID>-<task>.scope. These follow the naming convention set by systemd and also used by Docker when cgroups v2 is detected. Client nodes now export a new fingerprint attribute, unique.cgroups.version which will be set to 'v1' or 'v2' to indicate the cgroups regime in use by Nomad. The new cpuset management strategy fixes #11705, where docker tasks that spawned processes on startup would "leak". In cgroups v2, the PIDs are started in the cgroup they will always live in, and thus the cause of the leak is eliminated. [1] https://www.kernel.org/doc/html/latest/admin-guide/cgroup-v2.html Closes #11289 Fixes #11705 #11773 #11933
2022-02-28 22:24:01 +00:00
"github.com/hashicorp/nomad/client/lib/cgutil"
"github.com/hashicorp/nomad/client/lib/resources"
"github.com/hashicorp/nomad/client/stats"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/drivers/shared/capabilities"
shelpers "github.com/hashicorp/nomad/helper/stats"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
"github.com/opencontainers/runc/libcontainer"
"github.com/opencontainers/runc/libcontainer/cgroups"
lconfigs "github.com/opencontainers/runc/libcontainer/configs"
"github.com/opencontainers/runc/libcontainer/devices"
ldevices "github.com/opencontainers/runc/libcontainer/devices"
2020-08-19 15:57:26 +00:00
"github.com/opencontainers/runc/libcontainer/specconv"
lutils "github.com/opencontainers/runc/libcontainer/utils"
"github.com/opencontainers/runtime-spec/specs-go"
"golang.org/x/sys/unix"
)
var (
// ExecutorCgroupV1MeasuredMemStats is the list of memory stats captured by the executor with cgroup-v1
ExecutorCgroupV1MeasuredMemStats = []string{"RSS", "Cache", "Swap", "Usage", "Max Usage", "Kernel Usage", "Kernel Max Usage"}
// ExecutorCgroupV2MeasuredMemStats is the list of memory stats captured by the executor with cgroup-v2. cgroup-v2 exposes different memory stats and no longer reports rss or max usage.
ExecutorCgroupV2MeasuredMemStats = []string{"Cache", "Swap", "Usage"}
// ExecutorCgroupMeasuredCpuStats is the list of CPU stats captures by the executor
ExecutorCgroupMeasuredCpuStats = []string{"System Mode", "User Mode", "Throttled Periods", "Throttled Time", "Percent"}
2016-02-02 21:38:38 +00:00
)
// LibcontainerExecutor implements an Executor with the runc/libcontainer api
type LibcontainerExecutor struct {
id string
2018-12-07 01:54:14 +00:00
command *ExecCommand
logger hclog.Logger
totalCpuStats *stats.CpuStats
userCpuStats *stats.CpuStats
systemCpuStats *stats.CpuStats
pidCollector *pidCollector
container libcontainer.Container
userProc *libcontainer.Process
userProcExited chan interface{}
exitState *ProcessState
}
2018-12-07 01:54:14 +00:00
func NewExecutorWithIsolation(logger hclog.Logger) Executor {
logger = logger.Named("isolated_executor")
if err := shelpers.Init(); err != nil {
logger.Error("unable to initialize stats", "error", err)
}
return &LibcontainerExecutor{
id: strings.ReplaceAll(uuid.Generate(), "-", "_"),
logger: logger,
totalCpuStats: stats.NewCpuStats(),
userCpuStats: stats.NewCpuStats(),
systemCpuStats: stats.NewCpuStats(),
pidCollector: newPidCollector(logger),
2016-02-03 19:41:49 +00:00
}
2016-02-04 00:03:43 +00:00
}
2016-02-03 19:41:49 +00:00
// Launch creates a new container in libcontainer and starts a new process with it
2018-12-07 01:54:14 +00:00
func (l *LibcontainerExecutor) Launch(command *ExecCommand) (*ProcessState, error) {
l.logger.Trace("preparing to launch command", "command", command.Cmd, "args", strings.Join(command.Args, " "))
2018-10-16 02:37:58 +00:00
if command.Resources == nil {
command.Resources = &drivers.Resources{
2018-12-14 00:21:41 +00:00
NomadResources: &structs.AllocatedTaskResources{},
}
2018-10-16 02:37:58 +00:00
}
l.command = command
// create a new factory which will store the container state in the allocDir
factory, err := libcontainer.New(
path.Join(command.TaskDir, "../alloc/container"),
2019-04-19 13:40:30 +00:00
// note that os.Args[0] refers to the executor shim typically
// and first args arguments is ignored now due
// until https://github.com/opencontainers/runc/pull/1888 is merged
libcontainer.InitArgs(os.Args[0], "libcontainer-shim"),
)
if err != nil {
return nil, fmt.Errorf("failed to create factory: %v", err)
}
// A container groups processes under the same isolation enforcement
containerCfg, err := newLibcontainerConfig(command)
if err != nil {
return nil, fmt.Errorf("failed to configure container(%s): %v", l.id, err)
}
container, err := factory.Create(l.id, containerCfg)
if err != nil {
return nil, fmt.Errorf("failed to create container(%s): %v", l.id, err)
}
l.container = container
2018-10-16 02:37:58 +00:00
// Look up the binary path and make it executable
taskPath, hostPath, err := lookupTaskBin(command)
2018-10-16 02:37:58 +00:00
if err != nil {
return nil, err
}
if err := makeExecutable(hostPath); err != nil {
2018-10-16 02:37:58 +00:00
return nil, err
}
combined := append([]string{taskPath}, command.Args...)
stdout, err := command.Stdout()
if err != nil {
return nil, err
}
stderr, err := command.Stderr()
if err != nil {
return nil, err
}
l.logger.Debug("launching", "command", command.Cmd, "args", strings.Join(command.Args, " "))
// the task process will be started by the container
process := &libcontainer.Process{
Args: combined,
Env: command.Env,
Stdout: stdout,
Stderr: stderr,
Init: true,
}
if command.User != "" {
process.User = command.User
}
l.userProc = process
l.totalCpuStats = stats.NewCpuStats()
l.userCpuStats = stats.NewCpuStats()
l.systemCpuStats = stats.NewCpuStats()
// Starts the task
if err := container.Run(process); err != nil {
container.Destroy()
return nil, err
}
pid, err := process.Pid()
if err != nil {
container.Destroy()
return nil, err
}
// start a goroutine to wait on the process to complete, so Wait calls can
// be multiplexed
l.userProcExited = make(chan interface{})
go l.pidCollector.collectPids(l.userProcExited, l.getAllPids)
go l.wait()
2018-12-07 01:54:14 +00:00
return &ProcessState{
Pid: pid,
ExitCode: -1,
Time: time.Now(),
}, nil
}
func (l *LibcontainerExecutor) getAllPids() (resources.PIDs, error) {
pids, err := l.container.Processes()
if err != nil {
return nil, err
}
m := make(resources.PIDs, 1)
for _, pid := range pids {
m[pid] = resources.NewPID(pid)
}
return m, nil
}
// Wait waits until a process has exited and returns it's exitcode and errors
2018-12-07 01:54:14 +00:00
func (l *LibcontainerExecutor) Wait(ctx context.Context) (*ProcessState, error) {
select {
case <-ctx.Done():
return nil, ctx.Err()
case <-l.userProcExited:
return l.exitState, nil
}
}
func (l *LibcontainerExecutor) wait() {
defer close(l.userProcExited)
ps, err := l.userProc.Wait()
if err != nil {
// If the process has exited before we called wait an error is returned
// the process state is embedded in the error
if exitErr, ok := err.(*exec.ExitError); ok {
ps = exitErr.ProcessState
} else {
l.logger.Error("failed to call wait on user process", "error", err)
l.exitState = &ProcessState{Pid: 0, ExitCode: 1, Time: time.Now()}
return
}
}
l.command.Close()
exitCode := 1
var signal int
if status, ok := ps.Sys().(syscall.WaitStatus); ok {
exitCode = status.ExitStatus()
if status.Signaled() {
const exitSignalBase = 128
signal = int(status.Signal())
exitCode = exitSignalBase + signal
}
}
2018-12-07 01:54:14 +00:00
l.exitState = &ProcessState{
Pid: ps.Pid(),
ExitCode: exitCode,
Signal: signal,
Time: time.Now(),
}
}
// Shutdown stops all processes started and cleans up any resources
// created (such as mountpoints, devices, etc).
func (l *LibcontainerExecutor) Shutdown(signal string, grace time.Duration) error {
if l.container == nil {
2018-05-25 22:25:23 +00:00
return nil
}
status, err := l.container.Status()
if err != nil {
return err
}
defer l.container.Destroy()
if status == libcontainer.Stopped {
return nil
}
if grace > 0 {
if signal == "" {
signal = "SIGINT"
}
2016-02-04 00:03:43 +00:00
sig, ok := signals.SignalLookup[signal]
if !ok {
return fmt.Errorf("error unknown signal given for shutdown: %s", signal)
}
// Signal initial container processes only during graceful
// shutdown; hence `false` arg.
err = l.container.Signal(sig, false)
if err != nil {
return err
}
select {
case <-l.userProcExited:
return nil
case <-time.After(grace):
// Force kill all container processes after grace period,
// hence `true` argument.
if err := l.container.Signal(os.Kill, true); err != nil {
return err
}
}
} else {
err := l.container.Signal(os.Kill, true)
if err != nil {
return err
}
}
select {
case <-l.userProcExited:
return nil
case <-time.After(time.Second * 15):
return fmt.Errorf("process failed to exit after 15 seconds")
}
}
// UpdateResources updates the resource isolation with new values to be enforced
func (l *LibcontainerExecutor) UpdateResources(resources *drivers.Resources) error {
return nil
}
// Version returns the api version of the executor
2018-12-07 01:54:14 +00:00
func (l *LibcontainerExecutor) Version() (*ExecutorVersion, error) {
return &ExecutorVersion{Version: ExecutorVersionLatest}, nil
}
// Stats returns the resource statistics for processes managed by the executor
func (l *LibcontainerExecutor) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
ch := make(chan *cstructs.TaskResourceUsage)
go l.handleStats(ch, ctx, interval)
return ch, nil
}
2016-04-29 18:40:37 +00:00
func (l *LibcontainerExecutor) handleStats(ch chan *cstructs.TaskResourceUsage, ctx context.Context, interval time.Duration) {
defer close(ch)
timer := time.NewTimer(0)
measuredMemStats := ExecutorCgroupV1MeasuredMemStats
if cgroups.IsCgroup2UnifiedMode() {
measuredMemStats = ExecutorCgroupV2MeasuredMemStats
}
for {
select {
case <-ctx.Done():
return
case <-timer.C:
timer.Reset(interval)
}
lstats, err := l.container.Stats()
if err != nil {
l.logger.Warn("error collecting stats", "error", err)
return
}
pidStats, err := l.pidCollector.pidStats()
if err != nil {
l.logger.Warn("error collecting stats", "error", err)
return
}
ts := time.Now()
stats := lstats.CgroupStats
// Memory Related Stats
swap := stats.MemoryStats.SwapUsage
maxUsage := stats.MemoryStats.Usage.MaxUsage
rss := stats.MemoryStats.Stats["rss"]
cache := stats.MemoryStats.Stats["cache"]
mapped_file := stats.MemoryStats.Stats["mapped_file"]
ms := &cstructs.MemoryStats{
RSS: rss,
Cache: cache,
Swap: swap.Usage,
MappedFile: mapped_file,
Usage: stats.MemoryStats.Usage.Usage,
MaxUsage: maxUsage,
KernelUsage: stats.MemoryStats.KernelUsage.Usage,
KernelMaxUsage: stats.MemoryStats.KernelUsage.MaxUsage,
Measured: measuredMemStats,
}
// CPU Related Stats
totalProcessCPUUsage := float64(stats.CpuStats.CpuUsage.TotalUsage)
userModeTime := float64(stats.CpuStats.CpuUsage.UsageInUsermode)
kernelModeTime := float64(stats.CpuStats.CpuUsage.UsageInKernelmode)
totalPercent := l.totalCpuStats.Percent(totalProcessCPUUsage)
cs := &cstructs.CpuStats{
SystemMode: l.systemCpuStats.Percent(kernelModeTime),
UserMode: l.userCpuStats.Percent(userModeTime),
Percent: totalPercent,
ThrottledPeriods: stats.CpuStats.ThrottlingData.ThrottledPeriods,
ThrottledTime: stats.CpuStats.ThrottlingData.ThrottledTime,
TotalTicks: l.systemCpuStats.TicksConsumed(totalPercent),
Measured: ExecutorCgroupMeasuredCpuStats,
}
taskResUsage := cstructs.TaskResourceUsage{
ResourceUsage: &cstructs.ResourceUsage{
MemoryStats: ms,
CpuStats: cs,
},
Timestamp: ts.UTC().UnixNano(),
Pids: pidStats,
}
select {
case <-ctx.Done():
return
case ch <- &taskResUsage:
}
}
2016-04-28 23:06:01 +00:00
}
// Signal sends a signal to the process managed by the executor
func (l *LibcontainerExecutor) Signal(s os.Signal) error {
return l.userProc.Signal(s)
}
// Exec starts an additional process inside the container
func (l *LibcontainerExecutor) Exec(deadline time.Time, cmd string, args []string) ([]byte, int, error) {
combined := append([]string{cmd}, args...)
// Capture output
buf, _ := circbuf.NewBuffer(int64(drivers.CheckBufSize))
process := &libcontainer.Process{
Args: combined,
Env: l.command.Env,
Stdout: buf,
Stderr: buf,
}
err := l.container.Run(process)
if err != nil {
return nil, 0, err
}
waitCh := make(chan *waitResult)
defer close(waitCh)
go l.handleExecWait(waitCh, process)
select {
case result := <-waitCh:
ps := result.ps
if result.err != nil {
if exitErr, ok := result.err.(*exec.ExitError); ok {
ps = exitErr.ProcessState
} else {
return nil, 0, result.err
}
}
var exitCode int
if status, ok := ps.Sys().(syscall.WaitStatus); ok {
exitCode = status.ExitStatus()
}
return buf.Bytes(), exitCode, nil
case <-time.After(time.Until(deadline)):
process.Signal(os.Kill)
return nil, 0, context.DeadlineExceeded
}
}
func (l *LibcontainerExecutor) newTerminalSocket() (pty func() (*os.File, error), tty *os.File, err error) {
parent, child, err := lutils.NewSockPair("socket")
if err != nil {
return nil, nil, fmt.Errorf("failed to create terminal: %v", err)
}
return func() (*os.File, error) { return lutils.RecvFd(parent) }, child, err
}
func (l *LibcontainerExecutor) ExecStreaming(ctx context.Context, cmd []string, tty bool,
stream drivers.ExecTaskStream) error {
// the task process will be started by the container
process := &libcontainer.Process{
Args: cmd,
Env: l.userProc.Env,
User: l.userProc.User,
Init: false,
Cwd: "/",
}
execHelper := &execHelper{
logger: l.logger,
newTerminal: l.newTerminalSocket,
setTTY: func(tty *os.File) error {
process.ConsoleSocket = tty
return nil
},
setIO: func(stdin io.Reader, stdout, stderr io.Writer) error {
process.Stdin = stdin
process.Stdout = stdout
process.Stderr = stderr
return nil
},
processStart: func() error { return l.container.Run(process) },
processWait: func() (*os.ProcessState, error) {
return process.Wait()
},
}
return execHelper.run(ctx, tty, stream)
}
type waitResult struct {
ps *os.ProcessState
err error
}
func (l *LibcontainerExecutor) handleExecWait(ch chan *waitResult, process *libcontainer.Process) {
ps, err := process.Wait()
ch <- &waitResult{ps, err}
}
func configureCapabilities(cfg *lconfigs.Config, command *ExecCommand) {
switch command.User {
case "root":
// when running as root, use the legacy set of system capabilities, so
// that we do not break existing nomad clusters using this "feature"
legacyCaps := capabilities.LegacySupported().Slice(true)
2019-05-24 18:03:26 +00:00
cfg.Capabilities = &lconfigs.Capabilities{
Bounding: legacyCaps,
Permitted: legacyCaps,
Effective: legacyCaps,
2019-05-24 18:03:26 +00:00
Ambient: nil,
Inheritable: nil,
}
default:
// otherwise apply the plugin + task capability configuration
2019-05-24 18:03:26 +00:00
cfg.Capabilities = &lconfigs.Capabilities{
Bounding: command.Capabilities,
2019-05-24 18:03:26 +00:00
}
}
}
func configureNamespaces(pidMode, ipcMode string) lconfigs.Namespaces {
namespaces := lconfigs.Namespaces{{Type: lconfigs.NEWNS}}
2021-02-08 16:58:44 +00:00
if pidMode == IsolationModePrivate {
namespaces = append(namespaces, lconfigs.Namespace{Type: lconfigs.NEWPID})
}
2021-02-08 16:58:44 +00:00
if ipcMode == IsolationModePrivate {
namespaces = append(namespaces, lconfigs.Namespace{Type: lconfigs.NEWIPC})
}
return namespaces
}
// configureIsolation prepares the isolation primitives of the container.
// The process runs in a container configured with the following:
//
// * the task directory as the chroot
// * dedicated mount points namespace, but shares the PID, User, domain, network namespaces with host
// * small subset of devices (e.g. stdout/stderr/stdin, tty, shm, pts); default to using the same set of devices as Docker
// * some special filesystems: `/proc`, `/sys`. Some case is given to avoid exec escaping or setting malicious values through them.
func configureIsolation(cfg *lconfigs.Config, command *ExecCommand) error {
defaultMountFlags := syscall.MS_NOEXEC | syscall.MS_NOSUID | syscall.MS_NODEV
// set the new root directory for the container
cfg.Rootfs = command.TaskDir
// disable pivot_root if set in the driver's configuration
cfg.NoPivotRoot = command.NoPivotRoot
2021-02-08 16:58:44 +00:00
// set up default namespaces as configured
cfg.Namespaces = configureNamespaces(command.ModePID, command.ModeIPC)
if command.NetworkIsolation != nil {
cfg.Namespaces = append(cfg.Namespaces, lconfigs.Namespace{
Type: lconfigs.NEWNET,
Path: command.NetworkIsolation.Path,
})
}
// paths to mask using a bind mount to /dev/null to prevent reading
cfg.MaskPaths = []string{
"/proc/kcore",
"/sys/firmware",
}
// paths that should be remounted as readonly inside the container
cfg.ReadonlyPaths = []string{
"/proc/sys", "/proc/sysrq-trigger", "/proc/irq", "/proc/bus",
}
2020-08-19 15:57:26 +00:00
cfg.Devices = specconv.AllowedDevices
if len(command.Devices) > 0 {
devs, err := cmdDevices(command.Devices)
if err != nil {
return err
}
cfg.Devices = append(cfg.Devices, devs...)
}
cfg.Mounts = []*lconfigs.Mount{
{
Source: "tmpfs",
Destination: "/dev",
Device: "tmpfs",
Flags: syscall.MS_NOSUID | syscall.MS_STRICTATIME,
Data: "mode=755",
},
{
Source: "proc",
Destination: "/proc",
Device: "proc",
Flags: defaultMountFlags,
},
{
Source: "devpts",
Destination: "/dev/pts",
Device: "devpts",
Flags: syscall.MS_NOSUID | syscall.MS_NOEXEC,
Data: "newinstance,ptmxmode=0666,mode=0620,gid=5",
},
{
Device: "tmpfs",
Source: "shm",
Destination: "/dev/shm",
Data: "mode=1777,size=65536k",
Flags: defaultMountFlags,
},
{
Source: "mqueue",
Destination: "/dev/mqueue",
Device: "mqueue",
Flags: defaultMountFlags,
},
{
Source: "sysfs",
Destination: "/sys",
Device: "sysfs",
Flags: defaultMountFlags | syscall.MS_RDONLY,
},
}
if len(command.Mounts) > 0 {
cfg.Mounts = append(cfg.Mounts, cmdMounts(command.Mounts)...)
}
return nil
}
2018-12-07 01:54:14 +00:00
func configureCgroups(cfg *lconfigs.Config, command *ExecCommand) error {
// If resources are not limited then manually create cgroups needed
if !command.ResourceLimits {
client: enable support for cgroups v2 This PR introduces support for using Nomad on systems with cgroups v2 [1] enabled as the cgroups controller mounted on /sys/fs/cgroups. Newer Linux distros like Ubuntu 21.10 are shipping with cgroups v2 only, causing problems for Nomad users. Nomad mostly "just works" with cgroups v2 due to the indirection via libcontainer, but not so for managing cpuset cgroups. Before, Nomad has been making use of a feature in v1 where a PID could be a member of more than one cgroup. In v2 this is no longer possible, and so the logic around computing cpuset values must be modified. When Nomad detects v2, it manages cpuset values in-process, rather than making use of cgroup heirarchy inheritence via shared/reserved parents. Nomad will only activate the v2 logic when it detects cgroups2 is mounted at /sys/fs/cgroups. This means on systems running in hybrid mode with cgroups2 mounted at /sys/fs/cgroups/unified (as is typical) Nomad will continue to use the v1 logic, and should operate as before. Systems that do not support cgroups v2 are also not affected. When v2 is activated, Nomad will create a parent called nomad.slice (unless otherwise configured in Client conifg), and create cgroups for tasks using naming convention <allocID>-<task>.scope. These follow the naming convention set by systemd and also used by Docker when cgroups v2 is detected. Client nodes now export a new fingerprint attribute, unique.cgroups.version which will be set to 'v1' or 'v2' to indicate the cgroups regime in use by Nomad. The new cpuset management strategy fixes #11705, where docker tasks that spawned processes on startup would "leak". In cgroups v2, the PIDs are started in the cgroup they will always live in, and thus the cause of the leak is eliminated. [1] https://www.kernel.org/doc/html/latest/admin-guide/cgroup-v2.html Closes #11289 Fixes #11705 #11773 #11933
2022-02-28 22:24:01 +00:00
return cgutil.ConfigureBasicCgroups(cfg)
}
2016-02-04 00:03:43 +00:00
client: enable support for cgroups v2 This PR introduces support for using Nomad on systems with cgroups v2 [1] enabled as the cgroups controller mounted on /sys/fs/cgroups. Newer Linux distros like Ubuntu 21.10 are shipping with cgroups v2 only, causing problems for Nomad users. Nomad mostly "just works" with cgroups v2 due to the indirection via libcontainer, but not so for managing cpuset cgroups. Before, Nomad has been making use of a feature in v1 where a PID could be a member of more than one cgroup. In v2 this is no longer possible, and so the logic around computing cpuset values must be modified. When Nomad detects v2, it manages cpuset values in-process, rather than making use of cgroup heirarchy inheritence via shared/reserved parents. Nomad will only activate the v2 logic when it detects cgroups2 is mounted at /sys/fs/cgroups. This means on systems running in hybrid mode with cgroups2 mounted at /sys/fs/cgroups/unified (as is typical) Nomad will continue to use the v1 logic, and should operate as before. Systems that do not support cgroups v2 are also not affected. When v2 is activated, Nomad will create a parent called nomad.slice (unless otherwise configured in Client conifg), and create cgroups for tasks using naming convention <allocID>-<task>.scope. These follow the naming convention set by systemd and also used by Docker when cgroups v2 is detected. Client nodes now export a new fingerprint attribute, unique.cgroups.version which will be set to 'v1' or 'v2' to indicate the cgroups regime in use by Nomad. The new cpuset management strategy fixes #11705, where docker tasks that spawned processes on startup would "leak". In cgroups v2, the PIDs are started in the cgroup they will always live in, and thus the cause of the leak is eliminated. [1] https://www.kernel.org/doc/html/latest/admin-guide/cgroup-v2.html Closes #11289 Fixes #11705 #11773 #11933
2022-02-28 22:24:01 +00:00
// set cgroups path
if cgutil.UseV2 {
// in v2, the cgroup must have been created by the client already,
// which breaks a lot of existing tests that run drivers without a client
if command.Resources == nil || command.Resources.LinuxResources == nil || command.Resources.LinuxResources.CpusetCgroupPath == "" {
return errors.New("cgroup path must be set")
}
parent, cgroup := cgutil.SplitPath(command.Resources.LinuxResources.CpusetCgroupPath)
cfg.Cgroups.Path = filepath.Join("/", parent, cgroup)
} else {
// in v1, the cgroup is created using /nomad, which is a bug because it
// does not respect the cgroup_parent client configuration
// (but makes testing easy)
id := uuid.Generate()
cfg.Cgroups.Path = filepath.Join("/", cgutil.DefaultCgroupV1Parent, id)
}
if command.Resources == nil || command.Resources.NomadResources == nil {
return nil
}
// Total amount of memory allowed to consume
res := command.Resources.NomadResources
memHard, memSoft := res.Memory.MemoryMaxMB, res.Memory.MemoryMB
if memHard <= 0 {
memHard = res.Memory.MemoryMB
memSoft = 0
}
if memHard > 0 {
cfg.Cgroups.Resources.Memory = memHard * 1024 * 1024
cfg.Cgroups.Resources.MemoryReservation = memSoft * 1024 * 1024
// Disable swap to avoid issues on the machine
var memSwappiness uint64
cfg.Cgroups.Resources.MemorySwappiness = &memSwappiness
}
cpuShares := res.Cpu.CpuShares
2018-12-14 00:21:41 +00:00
if cpuShares < 2 {
return fmt.Errorf("resources.Cpu.CpuShares must be equal to or greater than 2: %v", cpuShares)
}
// Set the relative CPU shares for this cgroup, and convert for cgroupv2
2018-12-14 00:21:41 +00:00
cfg.Cgroups.Resources.CpuShares = uint64(cpuShares)
cfg.Cgroups.Resources.CpuWeight = cgroups.ConvertCPUSharesToCgroupV2Value(uint64(cpuShares))
if command.Resources.LinuxResources != nil && command.Resources.LinuxResources.CpusetCgroupPath != "" {
cfg.Hooks = lconfigs.Hooks{
lconfigs.CreateRuntime: lconfigs.HookList{
2021-04-16 19:14:29 +00:00
newSetCPUSetCgroupHook(command.Resources.LinuxResources.CpusetCgroupPath),
},
}
}
return nil
}
func newLibcontainerConfig(command *ExecCommand) (*lconfigs.Config, error) {
cfg := &lconfigs.Config{
Cgroups: &lconfigs.Cgroup{
Resources: &lconfigs.Resources{
MemorySwappiness: nil,
},
},
Version: "1.0.0",
}
2020-08-19 15:57:26 +00:00
for _, device := range specconv.AllowedDevices {
cfg.Cgroups.Resources.Devices = append(cfg.Cgroups.Resources.Devices, &device.Rule)
2020-08-19 15:57:26 +00:00
}
configureCapabilities(cfg, command)
// children should not inherit Nomad agent oom_score_adj value
oomScoreAdj := 0
cfg.OomScoreAdj = &oomScoreAdj
if err := configureIsolation(cfg, command); err != nil {
return nil, err
}
if err := configureCgroups(cfg, command); err != nil {
return nil, err
}
return cfg, nil
}
// cmdDevices converts a list of driver.DeviceConfigs into excutor.Devices.
func cmdDevices(driverDevices []*drivers.DeviceConfig) ([]*devices.Device, error) {
if len(driverDevices) == 0 {
return nil, nil
}
r := make([]*devices.Device, len(driverDevices))
for i, d := range driverDevices {
ed, err := ldevices.DeviceFromPath(d.HostPath, d.Permissions)
if err != nil {
return nil, fmt.Errorf("failed to make device out for %s: %v", d.HostPath, err)
}
ed.Path = d.TaskPath
r[i] = ed
}
return r, nil
}
var userMountToUnixMount = map[string]int{
// Empty string maps to `rprivate` for backwards compatibility in restored
// older tasks, where mount propagation will not be present.
"": unix.MS_PRIVATE | unix.MS_REC, // rprivate
structs.VolumeMountPropagationPrivate: unix.MS_PRIVATE | unix.MS_REC, // rprivate
structs.VolumeMountPropagationHostToTask: unix.MS_SLAVE | unix.MS_REC, // rslave
structs.VolumeMountPropagationBidirectional: unix.MS_SHARED | unix.MS_REC, // rshared
}
// cmdMounts converts a list of driver.MountConfigs into excutor.Mounts.
func cmdMounts(mounts []*drivers.MountConfig) []*lconfigs.Mount {
if len(mounts) == 0 {
return nil
}
r := make([]*lconfigs.Mount, len(mounts))
for i, m := range mounts {
flags := unix.MS_BIND
if m.Readonly {
flags |= unix.MS_RDONLY
}
r[i] = &lconfigs.Mount{
Source: m.HostPath,
Destination: m.TaskPath,
Device: "bind",
Flags: flags,
PropagationFlags: []int{userMountToUnixMount[m.PropagationMode]},
}
}
return r
}
// lookupTaskBin finds the file `bin`, searching in order:
// - taskDir/local
// - taskDir
// - each mount, in order listed in the jobspec
// - a PATH-like search of usr/local/bin/, usr/bin/, and bin/ inside the taskDir
//
// Returns an absolute path inside the container that will get passed as arg[0]
// to the launched process, and the absolute path to that binary as seen by the
// host (these will be identical for binaries that don't come from mounts).
//
// See also executor.lookupBin for a version used by non-isolated drivers.
func lookupTaskBin(command *ExecCommand) (string, string, error) {
taskDir := command.TaskDir
bin := command.Cmd
// Check in the local directory
localDir := filepath.Join(taskDir, allocdir.TaskLocal)
taskPath, hostPath, err := getPathInTaskDir(command.TaskDir, localDir, bin)
if err == nil {
return taskPath, hostPath, nil
}
// Check at the root of the task's directory
taskPath, hostPath, err = getPathInTaskDir(command.TaskDir, command.TaskDir, bin)
if err == nil {
return taskPath, hostPath, nil
}
// Check in our mounts
for _, mount := range command.Mounts {
taskPath, hostPath, err = getPathInMount(mount.HostPath, mount.TaskPath, bin)
if err == nil {
return taskPath, hostPath, nil
}
}
// If there's a / in the binary's path, we can't fallback to a PATH search
if strings.Contains(bin, "/") {
return "", "", fmt.Errorf("file %s not found under path %s", bin, taskDir)
}
// look for a file using a PATH-style lookup inside the directory
// root. Similar to the stdlib's exec.LookPath except:
// - uses a restricted lookup PATH rather than the agent process's PATH env var.
// - does not require that the file is already executable (this will be ensured
// by the caller)
// - does not prevent using relative path as added to exec.LookPath in go1.19
// (this gets fixed-up in the caller)
// This is a fake PATH so that we're not using the agent's PATH
restrictedPaths := []string{"/usr/local/bin", "/usr/bin", "/bin"}
for _, dir := range restrictedPaths {
pathDir := filepath.Join(command.TaskDir, dir)
taskPath, hostPath, err = getPathInTaskDir(command.TaskDir, pathDir, bin)
if err == nil {
return taskPath, hostPath, nil
}
}
return "", "", fmt.Errorf("file %s not found under path", bin)
}
// getPathInTaskDir searches for the binary in the task directory and nested
// search directory. It returns the absolute path rooted inside the container
// and the absolute path on the host.
func getPathInTaskDir(taskDir, searchDir, bin string) (string, string, error) {
hostPath := filepath.Join(searchDir, bin)
err := filepathIsRegular(hostPath)
if err != nil {
return "", "", err
}
// Find the path relative to the task directory
rel, err := filepath.Rel(taskDir, hostPath)
if rel == "" || err != nil {
return "", "", fmt.Errorf(
"failed to determine relative path base=%q target=%q: %v",
taskDir, hostPath, err)
}
// Turn relative-to-taskdir path into re-rooted absolute path to avoid
// libcontainer trying to resolve the binary using $PATH.
// Do *not* use filepath.Join as it will translate ".."s returned by
// filepath.Rel. Prepending "/" will cause the path to be rooted in the
// chroot which is the desired behavior.
return filepath.Clean("/" + rel), hostPath, nil
}
// getPathInMount for the binary in the mount's host path, constructing the path
// considering that the bin path is rooted in the mount's task path and not its
// host path. It returns the absolute path rooted inside the container and the
// absolute path on the host.
func getPathInMount(mountHostPath, mountTaskPath, bin string) (string, string, error) {
// Find the path relative to the mount point in the task so that we can
// trim off any shared prefix when we search on the host path
mountRel, err := filepath.Rel(mountTaskPath, bin)
if mountRel == "" || err != nil {
return "", "", fmt.Errorf("path was not relative to the mount task path")
}
hostPath := filepath.Join(mountHostPath, mountRel)
err = filepathIsRegular(hostPath)
if err != nil {
return "", "", err
}
// Turn relative-to-taskdir path into re-rooted absolute path to avoid
// libcontainer trying to resolve the binary using $PATH.
// Do *not* use filepath.Join as it will translate ".."s returned by
// filepath.Rel. Prepending "/" will cause the path to be rooted in the
// chroot which is the desired behavior.
return filepath.Clean("/" + bin), hostPath, nil
}
// filepathIsRegular verifies that a filepath is a regular file (i.e. not a
// directory, socket, device, etc.)
func filepathIsRegular(path string) error {
f, err := os.Stat(path)
if err != nil {
return err
}
if !f.Mode().Type().IsRegular() {
return fmt.Errorf("path was not a regular file")
}
return nil
}
2021-04-16 19:14:29 +00:00
func newSetCPUSetCgroupHook(cgroupPath string) lconfigs.Hook {
return lconfigs.NewFunctionHook(func(state *specs.State) error {
return cgroups.WriteCgroupProc(cgroupPath, state.Pid)
})
}