Implement 'runsc kill --all'

In order to implement kill --all correctly, the Sentry needs
to track all tasks that belong to a given container. This change
introduces ContainerID to the task, that gets inherited by all
children. 'kill --all' then iterates over all tasks comparing the
ContainerID field to find all processes that need to be signalled.

PiperOrigin-RevId: 214841768
Change-Id: I693b2374be8692d88cc441ef13a0ae34abf73ac6
This commit is contained in:
Fabricio Voznika 2018-09-27 15:00:03 -07:00 committed by Shentubot
parent 68ac2ad1e1
commit 491faac03b
14 changed files with 371 additions and 79 deletions

View File

@ -83,6 +83,9 @@ type ExecArgs struct {
// FilePayload determines the files to give to the new process.
urpc.FilePayload
// ContainerID is the container for the process being executed.
ContainerID string
}
// Exec runs a new task.
@ -133,6 +136,7 @@ func (proc *Proc) execAsync(args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadI
UTSNamespace: proc.Kernel.RootUTSNamespace(),
IPCNamespace: proc.Kernel.RootIPCNamespace(),
AbstractSocketNamespace: proc.Kernel.RootAbstractSocketNamespace(),
ContainerID: args.ContainerID,
}
if initArgs.Root != nil {
// initArgs must hold a reference on Root. This ref is dropped
@ -182,7 +186,7 @@ type PsArgs struct {
// Ps provides a process listing for the running kernel.
func (proc *Proc) Ps(args *PsArgs, out *string) error {
var p []*Process
if e := Processes(proc.Kernel, &p); e != nil {
if e := Processes(proc.Kernel, "", &p); e != nil {
return e
}
if !args.JSON {
@ -258,8 +262,9 @@ func PrintPIDsJSON(pl []*Process) (string, error) {
return string(b), nil
}
// Processes retrieves information about processes running in the sandbox.
func Processes(k *kernel.Kernel, out *[]*Process) error {
// Processes retrieves information about processes running in the sandbox with
// the given container id. All processes are returned if 'containerID' is empty.
func Processes(k *kernel.Kernel, containerID string, out *[]*Process) error {
ts := k.TaskSet()
now := k.RealtimeClock().Now()
for _, tg := range ts.Root.ThreadGroups() {
@ -268,6 +273,9 @@ func Processes(k *kernel.Kernel, out *[]*Process) error {
if pid == 0 {
continue
}
if containerID != "" && containerID != tg.Leader().ContainerID() {
continue
}
ppid := kernel.ThreadID(0)
if tg.Leader().Parent() != nil {

View File

@ -524,6 +524,9 @@ type CreateProcessArgs struct {
// Anyone setting Root must donate a reference (i.e. increment it) to
// keep it alive until it is decremented by CreateProcess.
Root *fs.Dirent
// ContainerID is the container that the process belongs to.
ContainerID string
}
// NewContext returns a context.Context that represents the task that will be
@ -660,6 +663,7 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
UTSNamespace: args.UTSNamespace,
IPCNamespace: args.IPCNamespace,
AbstractSocketNamespace: args.AbstractSocketNamespace,
ContainerID: args.ContainerID,
}
t, err := k.tasks.NewTask(config)
if err != nil {
@ -818,6 +822,27 @@ func (k *Kernel) SendExternalSignal(info *arch.SignalInfo, context string) {
k.sendExternalSignal(info, context)
}
// SendContainerSignal sends the given signal to all processes inside the
// namespace that match the given container ID.
func (k *Kernel) SendContainerSignal(cid string, info *arch.SignalInfo) error {
k.extMu.Lock()
defer k.extMu.Unlock()
k.tasks.mu.RLock()
defer k.tasks.mu.RUnlock()
for t := range k.tasks.Root.tids {
if t == t.tg.leader && t.ContainerID() == cid {
t.tg.signalHandlers.mu.Lock()
defer t.tg.signalHandlers.mu.Unlock()
infoCopy := *info
if err := t.sendSignalLocked(&infoCopy, true /*group*/); err != nil {
return err
}
}
}
return nil
}
// FeatureSet returns the FeatureSet.
func (k *Kernel) FeatureSet() *cpuid.FeatureSet {
return k.featureSet

View File

@ -205,6 +205,13 @@ type Task struct {
// k is the Kernel that this task belongs to. The k pointer is immutable.
k *Kernel
// containerID has no equivalent in Linux; it's used by runsc to track all
// tasks that belong to a given containers since cgroups aren't implemented.
// It's inherited by the children, is immutable, and may be empty.
//
// NOTE: cgroups can be used to track this when implemented.
containerID string
// mu protects some of the following fields.
mu sync.Mutex `state:"nosave"`
@ -678,3 +685,8 @@ func (t *Task) MountNamespace() *fs.MountNamespace {
func (t *Task) AbstractSockets() *AbstractSocketNamespace {
return t.abstractSockets
}
// ContainerID returns t's container ID.
func (t *Task) ContainerID() string {
return t.containerID
}

View File

@ -258,6 +258,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
UTSNamespace: utsns,
IPCNamespace: ipcns,
AbstractSocketNamespace: t.abstractSockets,
ContainerID: t.ContainerID(),
}
if opts.NewThreadGroup {
cfg.Parent = t

View File

@ -77,6 +77,9 @@ type TaskConfig struct {
// AbstractSocketNamespace is the AbstractSocketNamespace of the new task.
AbstractSocketNamespace *AbstractSocketNamespace
// ContainerID is the container the new task belongs to.
ContainerID string
}
// NewTask creates a new task defined by cfg.
@ -124,6 +127,7 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
abstractSockets: cfg.AbstractSocketNamespace,
rseqCPU: -1,
futexWaiter: futex.NewWaiter(),
containerID: cfg.ContainerID,
}
t.endStopCond.L = &t.tg.signalHandlers.mu
t.ptraceTracer.Store((*Task)(nil))

View File

@ -174,10 +174,17 @@ func (cm *containerManager) StartRoot(cid *string, _ *struct{}) error {
return nil
}
// ProcessesArgs container arguments to Processes method.
type ProcessesArgs struct {
// CID restricts the result to processes belonging to
// the given container. Empty means all.
CID string
}
// Processes retrieves information about processes running in the sandbox.
func (cm *containerManager) Processes(_, out *[]*control.Process) error {
func (cm *containerManager) Processes(args *ProcessesArgs, out *[]*control.Process) error {
log.Debugf("containerManager.Processes")
return control.Processes(cm.l.k, out)
return control.Processes(cm.l.k, args.CID, out)
}
// StartArgs contains arguments to the Start method.
@ -326,19 +333,11 @@ func (cm *containerManager) Destroy(cid *string, _ *struct{}) error {
return nil
}
// ExecArgs contains arguments to Execute.
type ExecArgs struct {
control.ExecArgs
// CID is the ID of the container to exec in.
CID string
}
// ExecuteAsync starts running a command on a created or running sandbox. It
// returns the pid of the new process.
func (cm *containerManager) ExecuteAsync(args *ExecArgs, pid *int32) error {
func (cm *containerManager) ExecuteAsync(args *control.ExecArgs, pid *int32) error {
log.Debugf("containerManager.ExecuteAsync: %+v", args)
tgid, err := cm.l.executeAsync(&args.ExecArgs, args.CID)
tgid, err := cm.l.executeAsync(args)
if err != nil {
return err
}
@ -503,11 +502,15 @@ type SignalArgs struct {
// Signo is the signal to send to the process.
Signo int32
// All is set when signal should be sent to all processes in the container.
// When false, the signal is sent to the root container process only.
All bool
}
// Signal sends a signal to the init process of the container.
// TODO: Send signal to exec process.
func (cm *containerManager) Signal(args *SignalArgs, _ *struct{}) error {
log.Debugf("containerManager.Signal")
return cm.l.signal(args.CID, args.Signo)
return cm.l.signal(args.CID, args.Signo, args.All)
}

View File

@ -270,7 +270,7 @@ func New(id string, spec *specs.Spec, conf *Config, controllerFD, deviceFD int,
log.Infof("Panic signal set to %v(%d)", ps, conf.PanicSignal)
}
procArgs, err := newProcess(spec, creds, k)
procArgs, err := newProcess(id, spec, creds, k)
if err != nil {
return nil, fmt.Errorf("failed to create root process: %v", err)
}
@ -295,7 +295,7 @@ func New(id string, spec *specs.Spec, conf *Config, controllerFD, deviceFD int,
}
// newProcess creates a process that can be run with kernel.CreateProcess.
func newProcess(spec *specs.Spec, creds *auth.Credentials, k *kernel.Kernel) (kernel.CreateProcessArgs, error) {
func newProcess(id string, spec *specs.Spec, creds *auth.Credentials, k *kernel.Kernel) (kernel.CreateProcessArgs, error) {
// Create initial limits.
ls, err := createLimitSet(spec)
if err != nil {
@ -314,6 +314,7 @@ func newProcess(spec *specs.Spec, creds *auth.Credentials, k *kernel.Kernel) (ke
UTSNamespace: k.RootUTSNamespace(),
IPCNamespace: k.RootIPCNamespace(),
AbstractSocketNamespace: k.RootAbstractSocketNamespace(),
ContainerID: id,
}
return procArgs, nil
}
@ -465,7 +466,7 @@ func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config
// TODO New containers should be started in new PID namespaces
// when indicated by the spec.
procArgs, err := newProcess(spec, creds, l.k)
procArgs, err := newProcess(cid, spec, creds, l.k)
if err != nil {
return fmt.Errorf("failed to create new process: %v", err)
}
@ -525,14 +526,14 @@ func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config
return nil
}
func (l *Loader) executeAsync(args *control.ExecArgs, cid string) (kernel.ThreadID, error) {
func (l *Loader) executeAsync(args *control.ExecArgs) (kernel.ThreadID, error) {
// Get the container Root Dirent from the Task, since we must run this
// process with the same Root.
l.mu.Lock()
tg, ok := l.containerRootTGs[cid]
tg, ok := l.containerRootTGs[args.ContainerID]
l.mu.Unlock()
if !ok {
return 0, fmt.Errorf("cannot exec in container %q: no such container", cid)
return 0, fmt.Errorf("cannot exec in container %q: no such container", args.ContainerID)
}
tg.Leader().WithMuLocked(func(t *kernel.Task) {
args.Root = t.FSContext().RootDirectory()
@ -552,7 +553,7 @@ func (l *Loader) executeAsync(args *control.ExecArgs, cid string) (kernel.Thread
// later.
l.mu.Lock()
defer l.mu.Unlock()
eid := execID{cid: cid, pid: tgid}
eid := execID{cid: args.ContainerID, pid: tgid}
l.execProcesses[eid] = tg
log.Debugf("updated execProcesses: %v", l.execProcesses)
@ -671,8 +672,7 @@ func newEmptyNetworkStack(conf *Config, clock tcpip.Clock) (inet.Stack, error) {
}
}
// TODO: Support sending signal to all.
func (l *Loader) signal(cid string, signo int32) error {
func (l *Loader) signal(cid string, signo int32, all bool) error {
l.mu.Lock()
tg, ok := l.containerRootTGs[cid]
l.mu.Unlock()
@ -681,5 +681,13 @@ func (l *Loader) signal(cid string, signo int32) error {
}
si := arch.SignalInfo{Signo: signo}
if all {
// Pause the kernel to prevent new processes from being created while
// the signal is delivered. This prevents process leaks when SIGKILL is
// sent to the entire container.
l.k.Pause()
defer l.k.Unpause()
return l.k.SendContainerSignal(cid, &si)
}
return tg.Leader().SendSignal(&si)
}

View File

@ -29,7 +29,9 @@ import (
)
// Kill implements subcommands.Command for the "kill" command.
type Kill struct{}
type Kill struct {
all bool
}
// Name implements subcommands.Command.Name.
func (*Kill) Name() string {
@ -47,15 +49,12 @@ func (*Kill) Usage() string {
}
// SetFlags implements subcommands.Command.SetFlags.
func (*Kill) SetFlags(f *flag.FlagSet) {
// TODO: Implement this flag. It is defined here just to
// prevent runsc from crashing if it is passed.
var all bool
f.BoolVar(&all, "all", false, "send the specified signal to all processes inside the container")
func (k *Kill) SetFlags(f *flag.FlagSet) {
f.BoolVar(&k.all, "all", false, "send the specified signal to all processes inside the container")
}
// Execute implements subcommands.Command.Execute.
func (*Kill) Execute(_ context.Context, f *flag.FlagSet, args ...interface{}) subcommands.ExitStatus {
func (k *Kill) Execute(_ context.Context, f *flag.FlagSet, args ...interface{}) subcommands.ExitStatus {
if f.NArg() == 0 || f.NArg() > 2 {
f.Usage()
return subcommands.ExitUsageError
@ -83,7 +82,7 @@ func (*Kill) Execute(_ context.Context, f *flag.FlagSet, args ...interface{}) su
}
// TODO: Distinguish between already-exited containers and
// genuine errors.
if err := c.Signal(sig); err != nil {
if err := c.Signal(sig, k.all); err != nil {
Fatalf("%v", err)
}
return subcommands.ExitSuccess

View File

@ -53,6 +53,7 @@ go_test(
"//runsc/boot",
"//runsc/specutils",
"//runsc/test/testutil",
"@com_github_cenkalti_backoff//:go_default_library",
"@com_github_opencontainers_runtime-spec//specs-go:go_default_library",
"@org_golang_x_sys//unix:go_default_library",
],
@ -61,5 +62,8 @@ go_test(
go_binary(
name = "test_app",
srcs = ["test_app.go"],
deps = ["@com_github_google_subcommands//:go_default_library"],
deps = [
"//runsc/test/testutil",
"@com_github_google_subcommands//:go_default_library",
],
)

View File

@ -159,7 +159,7 @@ func Load(rootDir, id string) (*Container, error) {
} else if c.Status == Running {
// Container state should reflect the actual state of the application, so
// we don't consider gofer process here.
if err := c.Signal(syscall.Signal(0)); err != nil {
if err := c.Signal(syscall.Signal(0), false); err != nil {
c.changeStatus(Stopped)
}
}
@ -398,7 +398,8 @@ func (c *Container) Execute(args *control.ExecArgs) (int32, error) {
if err := c.requireStatus("execute in", Created, Running); err != nil {
return 0, err
}
return c.Sandbox.Execute(c.ID, args)
args.ContainerID = c.ID
return c.Sandbox.Execute(args)
}
// Event returns events for the container.
@ -453,13 +454,13 @@ func (c *Container) WaitPID(pid int32, clearStatus bool) (syscall.WaitStatus, er
// Signal sends the signal to the container.
// Signal returns an error if the container is already stopped.
// TODO: Distinguish different error types.
func (c *Container) Signal(sig syscall.Signal) error {
func (c *Container) Signal(sig syscall.Signal, all bool) error {
log.Debugf("Signal container %q: %v", c.ID, sig)
if err := c.requireStatus("signal", Running); err != nil {
return err
}
// TODO: Query the container for its state, then save it.
return c.Sandbox.Signal(c.ID, sig)
return c.Sandbox.Signal(c.ID, sig, all)
}
// Checkpoint sends the checkpoint call to the container.
@ -612,7 +613,7 @@ func (c *Container) waitForStopped() error {
b := backoff.WithContext(backoff.NewConstantBackOff(100*time.Millisecond), ctx)
op := func() error {
if c.isSandboxRunning() {
if err := c.Signal(syscall.Signal(0)); err == nil {
if err := c.Signal(syscall.Signal(0), false); err == nil {
return fmt.Errorf("container is still running")
}
}

View File

@ -30,6 +30,7 @@ import (
"testing"
"time"
"github.com/cenkalti/backoff"
specs "github.com/opencontainers/runtime-spec/specs-go"
"golang.org/x/sys/unix"
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
@ -49,21 +50,34 @@ func init() {
}
// waitForProcessList waits for the given process list to show up in the container.
func waitForProcessList(cont *Container, expected []*control.Process) error {
var got []*control.Process
for start := time.Now(); time.Now().Sub(start) < 10*time.Second; {
var err error
got, err = cont.Processes()
func waitForProcessList(cont *Container, want []*control.Process) error {
cb := func() error {
got, err := cont.Processes()
if err != nil {
return fmt.Errorf("error getting process data from container: %v", err)
err = fmt.Errorf("error getting process data from container: %v", err)
return &backoff.PermanentError{Err: err}
}
if procListsEqual(got, expected) {
return nil
if !procListsEqual(got, want) {
return fmt.Errorf("container got process list: %s, want: %s", procListToString(got), procListToString(want))
}
// Process might not have started, try again...
time.Sleep(10 * time.Millisecond)
return nil
}
return fmt.Errorf("container got process list: %s, want: %s", procListToString(got), procListToString(expected))
return testutil.Poll(cb, 5*time.Second)
}
func waitForProcessCount(cont *Container, want int) error {
cb := func() error {
pss, err := cont.Processes()
if err != nil {
err = fmt.Errorf("error getting process data from container: %v", err)
return &backoff.PermanentError{Err: err}
}
if got := len(pss); got != want {
return fmt.Errorf("wrong process count, got: %d, want: %d", got, want)
}
return nil
}
return testutil.Poll(cb, 5*time.Second)
}
// procListsEqual is used to check whether 2 Process lists are equal for all
@ -345,7 +359,7 @@ func TestLifecycle(t *testing.T) {
<-ch
time.Sleep(100 * time.Millisecond)
// Send the container a SIGTERM which will cause it to stop.
if err := c.Signal(syscall.SIGTERM); err != nil {
if err := c.Signal(syscall.SIGTERM, false); err != nil {
t.Fatalf("error sending signal %v to container: %v", syscall.SIGTERM, err)
}
// Wait for it to die.

View File

@ -16,6 +16,7 @@ package container
import (
"io/ioutil"
"math"
"os"
"path"
"path/filepath"
@ -91,11 +92,16 @@ func TestMultiContainerSanity(t *testing.T) {
// Check via ps that multiple processes are running.
expectedPL := []*control.Process{
{PID: 1, Cmd: "sleep"},
{PID: 2, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
t.Errorf("failed to wait for sleep to start: %v", err)
}
expectedPL = []*control.Process{
{PID: 2, Cmd: "sleep"},
}
if err := waitForProcessList(containers[1], expectedPL); err != nil {
t.Errorf("failed to wait for sleep to start: %v", err)
}
}
}
@ -134,10 +140,9 @@ func TestMultiContainerWait(t *testing.T) {
// Check via ps that multiple processes are running.
expectedPL := []*control.Process{
{PID: 1, Cmd: "sleep"},
{PID: 2, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
if err := waitForProcessList(containers[1], expectedPL); err != nil {
t.Errorf("failed to wait for sleep to start: %v", err)
}
@ -179,7 +184,10 @@ func TestMultiContainerWait(t *testing.T) {
// After Wait returns, ensure that the root container is running and
// the child has finished.
if err := waitForProcessList(containers[0], expectedPL[:1]); err != nil {
expectedPL = []*control.Process{
{PID: 1, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
t.Errorf("failed to wait for %q to start: %v", strings.Join(containers[0].Spec.Process.Args, " "), err)
}
}
@ -219,17 +227,16 @@ func TestExecWait(t *testing.T) {
containers = append(containers, cont)
}
// Check via ps that multiple processes are running.
// Check via ps that process is running.
expectedPL := []*control.Process{
{PID: 1, Cmd: "sleep"},
{PID: 2, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
if err := waitForProcessList(containers[1], expectedPL); err != nil {
t.Fatalf("failed to wait for sleep to start: %v", err)
}
// Wait for the second container to finish.
if err := waitForProcessList(containers[0], expectedPL[:1]); err != nil {
if err := waitForProcessCount(containers[1], 0); err != nil {
t.Fatalf("failed to wait for second container to stop: %v", err)
}
@ -256,7 +263,10 @@ func TestExecWait(t *testing.T) {
}
// Wait for the exec'd process to exit.
if err := waitForProcessList(containers[0], expectedPL[:1]); err != nil {
expectedPL = []*control.Process{
{PID: 1, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
t.Fatalf("failed to wait for second container to stop: %v", err)
}
@ -360,23 +370,25 @@ func TestMultiContainerSignal(t *testing.T) {
containers = append(containers, cont)
}
// Check via ps that multiple processes are running.
// Check via ps that container 1 process is running.
expectedPL := []*control.Process{
{PID: 1, Cmd: "sleep"},
{PID: 2, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
if err := waitForProcessList(containers[1], expectedPL); err != nil {
t.Errorf("failed to wait for sleep to start: %v", err)
}
// Kill process 2.
if err := containers[1].Signal(syscall.SIGKILL); err != nil {
if err := containers[1].Signal(syscall.SIGKILL, false); err != nil {
t.Errorf("failed to kill process 2: %v", err)
}
// Make sure process 1 is still running.
if err := waitForProcessList(containers[0], expectedPL[:1]); err != nil {
expectedPL = []*control.Process{
{PID: 1, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL); err != nil {
t.Errorf("failed to wait for sleep to start: %v", err)
}
@ -395,18 +407,18 @@ func TestMultiContainerSignal(t *testing.T) {
t.Errorf("error waiting for gofer to exit: %v", err)
}
// Make sure process 1 is still running.
if err := waitForProcessList(containers[0], expectedPL[:1]); err != nil {
if err := waitForProcessList(containers[0], expectedPL); err != nil {
t.Errorf("failed to wait for sleep to start: %v", err)
}
// Now that process 2 is gone, ensure we get an error trying to
// signal it again.
if err := containers[1].Signal(syscall.SIGKILL); err == nil {
if err := containers[1].Signal(syscall.SIGKILL, false); err == nil {
t.Errorf("container %q shouldn't exist, but we were able to signal it", containers[1].ID)
}
// Kill process 1.
if err := containers[0].Signal(syscall.SIGKILL); err != nil {
if err := containers[0].Signal(syscall.SIGKILL, false); err != nil {
t.Errorf("failed to kill process 1: %v", err)
}
@ -428,7 +440,7 @@ func TestMultiContainerSignal(t *testing.T) {
}
// The sentry should be gone, so signaling should yield an error.
if err := containers[0].Signal(syscall.SIGKILL); err == nil {
if err := containers[0].Signal(syscall.SIGKILL, false); err == nil {
t.Errorf("sandbox %q shouldn't exist, but we were able to signal it", containers[0].Sandbox.ID)
}
}
@ -453,7 +465,6 @@ func TestMultiContainerDestroy(t *testing.T) {
// Setup the containers.
var containers []*Container
for i, spec := range specs {
conf := testutil.TestConfig()
bundleDir, err := testutil.SetupContainerInRoot(rootDir, spec, conf)
if err != nil {
t.Fatalf("error setting up container: %v", err)
@ -501,3 +512,144 @@ func TestMultiContainerDestroy(t *testing.T) {
}
}
}
func TestMultiContainerProcesses(t *testing.T) {
// Note: use 'while true' to keep 'sh' process around. Otherwise, shell will
// just execve into 'sleep' and both containers will look the same.
specs, ids := createSpecs(
[]string{"sleep", "100"},
[]string{"sh", "-c", "while true; do sleep 100; done"})
rootDir, err := testutil.SetupRootDir()
if err != nil {
t.Fatalf("error creating root dir: %v", err)
}
defer os.RemoveAll(rootDir)
var containers []*Container
for i, spec := range specs {
conf := testutil.TestConfig()
bundleDir, err := testutil.SetupContainerInRoot(rootDir, spec, conf)
if err != nil {
t.Fatalf("error setting up container: %v", err)
}
defer os.RemoveAll(bundleDir)
cont, err := Create(ids[i], spec, conf, bundleDir, "", "")
if err != nil {
t.Fatalf("error creating container: %v", err)
}
defer cont.Destroy()
if err := cont.Start(conf); err != nil {
t.Fatalf("error starting container: %v", err)
}
containers = append(containers, cont)
}
// Check root's container process list doesn't include other containers.
expectedPL0 := []*control.Process{
{PID: 1, Cmd: "sleep"},
}
if err := waitForProcessList(containers[0], expectedPL0); err != nil {
t.Errorf("failed to wait for process to start: %v", err)
}
// Same for the other container.
expectedPL1 := []*control.Process{
{PID: 2, Cmd: "sh"},
{PID: 3, PPID: 2, Cmd: "sleep"},
}
if err := waitForProcessList(containers[1], expectedPL1); err != nil {
t.Errorf("failed to wait for process to start: %v", err)
}
// Now exec into the second container and verify it shows up in the container.
args := &control.ExecArgs{
Filename: "/bin/sleep",
Argv: []string{"/bin/sleep", "100"},
}
if _, err := containers[1].Execute(args); err != nil {
t.Fatalf("error exec'ing: %v", err)
}
expectedPL1 = append(expectedPL1, &control.Process{PID: 4, Cmd: "sleep"})
if err := waitForProcessList(containers[1], expectedPL1); err != nil {
t.Errorf("failed to wait for process to start: %v", err)
}
// Root container should remain unchanged.
if err := waitForProcessList(containers[0], expectedPL0); err != nil {
t.Errorf("failed to wait for process to start: %v", err)
}
}
// TestMultiContainerKillAll checks that all process that belong to a container
// are killed when SIGKILL is sent to *all* processes in that container.
func TestMultiContainerKillAll(t *testing.T) {
app, err := testutil.FindFile("runsc/container/test_app")
if err != nil {
t.Fatal("error finding test_app:", err)
}
// First container will remain intact while the second container is killed.
specs, ids := createSpecs(
[]string{app, "task-tree", "--depth=2", "--width=2"},
[]string{app, "task-tree", "--depth=4", "--width=2"})
rootDir, err := testutil.SetupRootDir()
if err != nil {
t.Fatalf("error creating root dir: %v", err)
}
defer os.RemoveAll(rootDir)
var containers []*Container
for i, spec := range specs {
conf := testutil.TestConfig()
bundleDir, err := testutil.SetupContainerInRoot(rootDir, spec, conf)
if err != nil {
t.Fatalf("error setting up container: %v", err)
}
defer os.RemoveAll(bundleDir)
cont, err := Create(ids[i], spec, conf, bundleDir, "", "")
if err != nil {
t.Fatalf("error creating container: %v", err)
}
defer cont.Destroy()
if err := cont.Start(conf); err != nil {
t.Fatalf("error starting container: %v", err)
}
containers = append(containers, cont)
}
// Wait until all processes are created.
rootProcCount := int(math.Pow(2, 3) - 1)
if err := waitForProcessCount(containers[0], rootProcCount); err != nil {
t.Fatal(err)
}
procCount := int(math.Pow(2, 5) - 1)
if err := waitForProcessCount(containers[1], procCount); err != nil {
t.Fatal(err)
}
// Exec more processes to ensure signal works for exec'd processes too.
args := &control.ExecArgs{
Filename: app,
Argv: []string{app, "task-tree", "--depth=2", "--width=2"},
}
if _, err := containers[1].Execute(args); err != nil {
t.Fatalf("error exec'ing: %v", err)
}
procCount += 3
if err := waitForProcessCount(containers[1], procCount); err != nil {
t.Fatal(err)
}
// Kill'Em All
containers[1].Signal(syscall.SIGKILL, true)
// Check that all processes are gone.
if err := waitForProcessCount(containers[1], 0); err != nil {
t.Fatal(err)
}
// Check that root container was not affected.
if err := waitForProcessCount(containers[0], rootProcCount); err != nil {
t.Fatal(err)
}
}

View File

@ -22,17 +22,20 @@ import (
"log"
"net"
"os"
"os/exec"
"strconv"
"time"
"flag"
"github.com/google/subcommands"
"gvisor.googlesource.com/gvisor/runsc/test/testutil"
)
func main() {
subcommands.Register(subcommands.HelpCommand(), "")
subcommands.Register(subcommands.FlagsCommand(), "")
subcommands.Register(new(uds), "")
subcommands.Register(new(taskTree), "")
flag.Parse()
@ -114,3 +117,63 @@ func server(listener net.Listener, out *os.File) {
fmt.Fprint(out, string(data)+"\n")
}
}
type taskTree struct {
depth int
width int
}
// Name implements subcommands.Command.
func (*taskTree) Name() string {
return "task-tree"
}
// Synopsis implements subcommands.Command.
func (*taskTree) Synopsis() string {
return "creates a tree of tasks"
}
// Usage implements subcommands.Command.
func (*taskTree) Usage() string {
return "task-tree <flags>"
}
// SetFlags implements subcommands.Command.
func (c *taskTree) SetFlags(f *flag.FlagSet) {
f.IntVar(&c.depth, "depth", 1, "number of levels to create")
f.IntVar(&c.width, "width", 1, "number of tasks at each level")
}
// Execute implements subcommands.Command.
func (c *taskTree) Execute(ctx context.Context, f *flag.FlagSet, args ...interface{}) subcommands.ExitStatus {
stop := testutil.StartReaper()
defer stop()
if c.depth == 0 {
log.Printf("Child sleeping, PID: %d\n", os.Getpid())
for {
time.Sleep(24 * time.Hour)
}
}
log.Printf("Parent %d sleeping, PID: %d\n", c.depth, os.Getpid())
var cmds []*exec.Cmd
for i := 0; i < c.width; i++ {
cmd := exec.Command(
"/proc/self/exe", c.Name(),
"--depth", strconv.Itoa(c.depth-1),
"--width", strconv.Itoa(c.width))
cmd.Stdout = os.Stdout
cmd.Stderr = os.Stderr
if err := cmd.Start(); err != nil {
log.Fatal("failed to call self:", err)
}
cmds = append(cmds, cmd)
}
for _, c := range cmds {
c.Wait()
}
return subcommands.ExitSuccess
}

View File

@ -183,10 +183,9 @@ func (s *Sandbox) Processes(cid string) ([]*control.Process, error) {
}
defer conn.Close()
args := boot.ProcessesArgs{CID: cid}
var pl []*control.Process
// TODO: Pass in the container id (cid) here. The sandbox
// should return process info for only that container.
if err := conn.Call(boot.ContainerProcesses, nil, &pl); err != nil {
if err := conn.Call(boot.ContainerProcesses, &args, &pl); err != nil {
return nil, fmt.Errorf("error retrieving process data from sandbox: %v", err)
}
return pl, nil
@ -194,19 +193,17 @@ func (s *Sandbox) Processes(cid string) ([]*control.Process, error) {
// Execute runs the specified command in the container. It returns the pid of
// the newly created process.
func (s *Sandbox) Execute(cid string, args *control.ExecArgs) (int32, error) {
log.Debugf("Executing new process in container %q in sandbox %q", cid, s.ID)
func (s *Sandbox) Execute(args *control.ExecArgs) (int32, error) {
log.Debugf("Executing new process in container %q in sandbox %q", args.ContainerID, s.ID)
conn, err := s.sandboxConnect()
if err != nil {
return 0, s.connError(err)
}
defer conn.Close()
rpcArgs := &boot.ExecArgs{ExecArgs: *args, CID: cid}
// Send a message to the sandbox control server to start the container.
var pid int32
if err := conn.Call(boot.ContainerExecuteAsync, rpcArgs, &pid); err != nil {
if err := conn.Call(boot.ContainerExecuteAsync, args, &pid); err != nil {
return 0, fmt.Errorf("error executing in sandbox: %v", err)
}
return pid, nil
@ -575,7 +572,7 @@ func (s *Sandbox) destroy() error {
}
// Signal sends the signal to a container in the sandbox.
func (s *Sandbox) Signal(cid string, sig syscall.Signal) error {
func (s *Sandbox) Signal(cid string, sig syscall.Signal, all bool) error {
log.Debugf("Signal sandbox %q", s.ID)
conn, err := s.sandboxConnect()
if err != nil {
@ -586,6 +583,7 @@ func (s *Sandbox) Signal(cid string, sig syscall.Signal) error {
args := boot.SignalArgs{
CID: cid,
Signo: int32(sig),
All: all,
}
if err := conn.Call(boot.ContainerSignal, &args, nil); err != nil {
return fmt.Errorf("err signaling container %q: %v", cid, err)