summaryrefslogtreecommitdiffhomepage
path: root/runsc
diff options
context:
space:
mode:
authorNicolas Lacasse <nlacasse@google.com>2018-10-17 12:27:58 -0700
committerShentubot <shentubot@google.com>2018-10-17 12:29:05 -0700
commit4e6f0892c96c374b1abcf5c39b75ba52d98c97f8 (patch)
treecb21538ad26a50ff61086d55c1bef36d5026e4c0 /runsc
parent578fe5a50dcf8e104b6bce3802987b0f8c069ade (diff)
runsc: Support job control signals for the root container.
Now containers run with "docker run -it" support control characters like ^C and ^Z. This required refactoring our signal handling a bit. Signals delivered to the "runsc boot" process are turned into loader.Signal calls with the appropriate delivery mode. Previously they were always sent directly to PID 1. PiperOrigin-RevId: 217566770 Change-Id: I5b7220d9a0f2b591a56335479454a200c6de8732
Diffstat (limited to 'runsc')
-rw-r--r--runsc/boot/controller.go58
-rw-r--r--runsc/boot/fds.go45
-rw-r--r--runsc/boot/loader.go221
-rw-r--r--runsc/container/BUILD1
-rw-r--r--runsc/container/console_test.go452
-rw-r--r--runsc/container/container_test.go203
-rw-r--r--runsc/sandbox/sandbox.go24
-rw-r--r--runsc/test/integration/exec_test.go2
-rw-r--r--runsc/test/integration/integration_test.go48
-rw-r--r--runsc/test/testutil/docker.go39
10 files changed, 740 insertions, 353 deletions
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go
index eaeb9e2d8..bee82f344 100644
--- a/runsc/boot/controller.go
+++ b/runsc/boot/controller.go
@@ -425,6 +425,26 @@ func (cm *containerManager) WaitPID(args *WaitPIDArgs, waitStatus *uint32) error
return cm.l.waitPID(kernel.ThreadID(args.PID), args.CID, args.ClearStatus, waitStatus)
}
+// SignalDeliveryMode enumerates different signal delivery modes.
+type SignalDeliveryMode int
+
+const (
+ // DeliverToProcess delivers the signal to the container process with
+ // the specified PID. If PID is 0, then the container init process is
+ // signaled.
+ DeliverToProcess SignalDeliveryMode = iota
+
+ // DeliverToAllProcesses delivers the signal to all processes in the
+ // container. PID must be 0.
+ DeliverToAllProcesses
+
+ // DeliverToForegroundProcessGroup delivers the signal to the
+ // foreground process group in the same TTY session as the specified
+ // process. If PID is 0, then the signal is delivered to the foreground
+ // process group for the TTY for the init process.
+ DeliverToForegroundProcessGroup
+)
+
// SignalArgs are arguments to the Signal method.
type SignalArgs struct {
// CID is the container ID.
@@ -433,36 +453,20 @@ 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 root process of the container.
-func (cm *containerManager) Signal(args *SignalArgs, _ *struct{}) error {
- log.Debugf("containerManager.Signal %q %d, all: %t", args.CID, args.Signo, args.All)
- return cm.l.signalContainer(args.CID, args.Signo, args.All)
-}
-
-// SignalProcessArgs are arguments to the Signal method.
-type SignalProcessArgs struct {
- // CID is the container ID.
- CID string
-
// PID is the process ID in the given container that will be signaled.
+ // If 0, the root container will be signalled.
PID int32
- // Signo is the signal to send to the process.
- Signo int32
-
- // SendToForegroundProcess indicates that the signal should be sent to
- // the foreground process group in the session that PID belongs to.
- // This is only valid if the process is attached to a host TTY.
- SendToForegroundProcess bool
+ // Mode is the signal delivery mode.
+ Mode SignalDeliveryMode
}
-// SignalProcess sends a signal to a particular process in the container.
-func (cm *containerManager) SignalProcess(args *SignalProcessArgs, _ *struct{}) error {
- log.Debugf("containerManager.Signal: %+v", args)
- return cm.l.signalProcess(args.CID, args.PID, args.Signo, args.SendToForegroundProcess)
+// Signal sends a signal to one or more processes in a container. If args.PID
+// is 0, then the container init process is used. Depending on the
+// args.SignalDeliveryMode option, the signal may be sent directly to the
+// indicated process, to all processes in the container, or to the foreground
+// process group.
+func (cm *containerManager) Signal(args *SignalArgs, _ *struct{}) error {
+ log.Debugf("containerManager.Signal %+v", args)
+ return cm.l.signal(args.CID, args.PID, args.Signo, args.Mode)
}
diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go
index a5a6ba8af..9416e3a5c 100644
--- a/runsc/boot/fds.go
+++ b/runsc/boot/fds.go
@@ -35,6 +35,7 @@ func createFDMap(ctx context.Context, k *kernel.Kernel, l *limits.LimitSet, cons
fdm := k.NewFDMap()
defer fdm.DecRef()
+ mounter := fs.FileOwnerFromContext(ctx)
// Maps sandbox FD to host FD.
fdMap := map[int]int{
@@ -42,16 +43,44 @@ func createFDMap(ctx context.Context, k *kernel.Kernel, l *limits.LimitSet, cons
1: stdioFDs[1],
2: stdioFDs[2],
}
- mounter := fs.FileOwnerFromContext(ctx)
- for sfd, hfd := range fdMap {
- file, err := host.ImportFile(ctx, hfd, mounter, console /* isTTY */)
- if err != nil {
- return nil, fmt.Errorf("failed to import fd %d: %v", hfd, err)
+ var ttyFile *fs.File
+ for appFD, hostFD := range fdMap {
+ var appFile *fs.File
+
+ if console && appFD < 3 {
+ // Import the file as a host TTY file.
+ if ttyFile == nil {
+ var err error
+ appFile, err = host.ImportFile(ctx, hostFD, mounter, true /* isTTY */)
+ if err != nil {
+ return nil, err
+ }
+ defer appFile.DecRef()
+
+ // Remember this in the TTY file, as we will
+ // use it for the other stdio FDs.
+ ttyFile = appFile
+ } else {
+ // Re-use the existing TTY file, as all three
+ // stdio FDs must point to the same fs.File in
+ // order to share TTY state, specifically the
+ // foreground process group id.
+ appFile = ttyFile
+ }
+ } else {
+ // Import the file as a regular host file.
+ var err error
+ appFile, err = host.ImportFile(ctx, hostFD, mounter, false /* isTTY */)
+ if err != nil {
+ return nil, err
+ }
+ defer appFile.DecRef()
}
- defer file.DecRef()
- if err := fdm.NewFDAt(kdefs.FD(sfd), file, kernel.FDFlags{}, l); err != nil {
- return nil, fmt.Errorf("failed to add imported fd %d to FDMap: %v", hfd, err)
+
+ // Add the file to the FD map.
+ if err := fdm.NewFDAt(kdefs.FD(appFD), appFile, kernel.FDFlags{}, l); err != nil {
+ return nil, err
}
}
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index fa169d090..c79b95bde 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -19,7 +19,6 @@ import (
"fmt"
mrand "math/rand"
"os"
- "os/signal"
"runtime"
"sync"
"sync/atomic"
@@ -110,7 +109,7 @@ type Loader struct {
// mu guards processes.
mu sync.Mutex
- // processes maps containers root process and invocation of exec. Root
+ // processes maps containers init process and invocation of exec. Root
// processes are keyed with container ID and pid=0, while exec invocations
// have the corresponding pid set.
//
@@ -291,28 +290,9 @@ func New(args Args) (*Loader, error) {
return nil, fmt.Errorf("error creating control server: %v", err)
}
- // We don't care about child signals; some platforms can generate a
- // tremendous number of useless ones (I'm looking at you, ptrace).
- if err := sighandling.IgnoreChildStop(); err != nil {
- return nil, fmt.Errorf("failed to ignore child stop signals: %v", err)
- }
- // Ensure that signals received are forwarded to the emulated kernel.
- ps := syscall.Signal(args.Conf.PanicSignal)
- startSignalForwarding := sighandling.PrepareForwarding(k, ps)
- if args.Conf.PanicSignal != -1 {
- // Panics if the sentry receives 'Config.PanicSignal'.
- panicChan := make(chan os.Signal, 1)
- signal.Notify(panicChan, ps)
- go func() { // S/R-SAFE: causes sentry panic.
- <-panicChan
- panic("Signal-induced panic")
- }()
- log.Infof("Panic signal set to %v(%d)", ps, args.Conf.PanicSignal)
- }
-
procArgs, err := newProcess(args.ID, args.Spec, creds, k)
if err != nil {
- return nil, fmt.Errorf("failed to create root process: %v", err)
+ return nil, fmt.Errorf("failed to create init process for root container: %v", err)
}
if err := initCompatLogs(args.UserLogFD); err != nil {
@@ -320,19 +300,47 @@ func New(args Args) (*Loader, error) {
}
l := &Loader{
- k: k,
- ctrl: ctrl,
- conf: args.Conf,
- console: args.Console,
- watchdog: watchdog,
- spec: args.Spec,
- goferFDs: args.GoferFDs,
- stdioFDs: args.StdioFDs,
- startSignalForwarding: startSignalForwarding,
- rootProcArgs: procArgs,
- sandboxID: args.ID,
- processes: make(map[execID]*execProcess),
+ k: k,
+ ctrl: ctrl,
+ conf: args.Conf,
+ console: args.Console,
+ watchdog: watchdog,
+ spec: args.Spec,
+ goferFDs: args.GoferFDs,
+ stdioFDs: args.StdioFDs,
+ rootProcArgs: procArgs,
+ sandboxID: args.ID,
+ processes: make(map[execID]*execProcess),
}
+
+ // We don't care about child signals; some platforms can generate a
+ // tremendous number of useless ones (I'm looking at you, ptrace).
+ if err := sighandling.IgnoreChildStop(); err != nil {
+ return nil, fmt.Errorf("failed to ignore child stop signals: %v", err)
+ }
+
+ // Handle signals by forwarding them to the root container process
+ // (except for panic signal, which should cause a panic).
+ l.startSignalForwarding = sighandling.PrepareHandler(func(sig linux.Signal) {
+ // Panic signal should cause a panic.
+ if args.Conf.PanicSignal != -1 && sig == linux.Signal(args.Conf.PanicSignal) {
+ panic("Signal-induced panic")
+ }
+
+ // Otherwise forward to root container.
+ deliveryMode := DeliverToProcess
+ if args.Console {
+ // Since we are running with a console, we should
+ // forward the signal to the foreground process group
+ // so that job control signals like ^C can be handled
+ // properly.
+ deliveryMode = DeliverToForegroundProcessGroup
+ }
+ if err := l.signal(args.ID, 0, int32(sig), deliveryMode); err != nil {
+ log.Warningf("error sending signal %v to container %q: %v", sig, args.ID, err)
+ }
+ })
+
ctrl.manager.l = l
return l, nil
}
@@ -467,9 +475,15 @@ func (l *Loader) run() error {
l.rootProcArgs.FDMap.DecRef()
}
- l.mu.Lock()
eid := execID{cid: l.sandboxID}
- l.processes[eid] = &execProcess{tg: l.k.GlobalInit()}
+ ep := execProcess{tg: l.k.GlobalInit()}
+ if l.console {
+ ttyFile := l.rootProcArgs.FDMap.GetFile(0)
+ defer ttyFile.DecRef()
+ ep.tty = ttyFile.FileOperations.(*host.TTYFileOperations)
+ }
+ l.mu.Lock()
+ l.processes[eid] = &ep
l.mu.Unlock()
// Start signal forwarding only after an init process is created.
@@ -572,7 +586,7 @@ func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config
// filesystem.
func (l *Loader) destroyContainer(cid string) error {
// First kill and wait for all processes in the container.
- if err := l.signalContainer(cid, int32(linux.SIGKILL), true /*all*/); err != nil {
+ if err := l.signal(cid, 0, int32(linux.SIGKILL), DeliverToAllProcesses); err != nil {
return fmt.Errorf("failed to SIGKILL all container processes: %v", err)
}
@@ -634,7 +648,7 @@ func (l *Loader) executeAsync(args *control.ExecArgs) (kernel.ThreadID, error) {
return tgid, nil
}
-// waitContainer waits for the root process of a container to exit.
+// waitContainer waits for the init process of a container to exit.
func (l *Loader) waitContainer(cid string, waitStatus *uint32) error {
// Don't defer unlock, as doing so would make it impossible for
// multiple clients to wait on the same container.
@@ -740,11 +754,12 @@ func newEmptyNetworkStack(conf *Config, clock tcpip.Clock) (inet.Stack, error) {
}
}
-// signalProcess sends a signal to the process with the given PID. If
-// sendToFGProcess is true, then the signal will be sent to the foreground
-// process group in the same session that PID belongs to.
-func (l *Loader) signalProcess(cid string, pid, signo int32, sendToFGProcess bool) error {
- if pid <= 0 {
+// signal sends a signal to one or more processes in a container. If PID is 0,
+// then the container init process is used. Depending on the SignalDeliveryMode
+// option, the signal may be sent directly to the indicated process, to all
+// processes in the container, or to the foreground process group.
+func (l *Loader) signal(cid string, pid, signo int32, mode SignalDeliveryMode) error {
+ if pid < 0 {
return fmt.Errorf("failed to signal container %q PID %d: PID must be positive", cid, pid)
}
@@ -756,10 +771,16 @@ func (l *Loader) signalProcess(cid string, pid, signo int32, sendToFGProcess boo
ep, ok := l.processes[eid]
l.mu.Unlock()
- // The caller may be signaling a process not started directly via exec.
- // In this case, find the process in the container's PID namespace and
- // signal it.
- if !ok {
+ switch mode {
+ case DeliverToProcess:
+ if ok {
+ // Send signal directly to the identified process.
+ return ep.tg.SendSignal(&arch.SignalInfo{Signo: signo})
+ }
+
+ // The caller may be signaling a process not started directly via exec.
+ // In this case, find the process in the container's PID namespace and
+ // signal it.
ep, ok := l.processes[execID{cid: cid}]
if !ok {
return fmt.Errorf("no container with ID: %q", cid)
@@ -772,74 +793,60 @@ func (l *Loader) signalProcess(cid string, pid, signo int32, sendToFGProcess boo
return fmt.Errorf("process %d is part of a different container: %q", pid, tg.Leader().ContainerID())
}
return tg.SendSignal(&arch.SignalInfo{Signo: signo})
- }
-
- if !sendToFGProcess {
- // Send signal directly to exec process.
- return ep.tg.SendSignal(&arch.SignalInfo{Signo: signo})
- }
- // Lookup foreground process group from the TTY for the given process,
- // and send the signal to it.
- if ep.tty == nil {
- return fmt.Errorf("failed to signal foreground process group in container %q PID %d: no TTY attached", cid, pid)
- }
- pg := ep.tty.ForegroundProcessGroup()
- if pg == nil {
- // No foreground process group has been set. Signal the
- // original thread group.
- log.Warningf("No foreground process group for container %q and PID %d. Sending signal directly to PID %d.", cid, pid, pid)
- return ep.tg.SendSignal(&arch.SignalInfo{Signo: signo})
- }
+ case DeliverToForegroundProcessGroup:
+ if !ok {
+ return fmt.Errorf("failed to signal foreground process group for container %q PID %d: no such PID", cid, pid)
+ }
- // Send the signal to all processes in the process group.
- var lastErr error
- for _, tg := range l.k.TaskSet().Root.ThreadGroups() {
- if tg.ProcessGroup() != pg {
- continue
+ // Lookup foreground process group from the TTY for the given process,
+ // and send the signal to it.
+ if ep.tty == nil {
+ return fmt.Errorf("failed to signal foreground process group in container %q PID %d: no TTY attached", cid, pid)
}
- if err := tg.SendSignal(&arch.SignalInfo{Signo: signo}); err != nil {
- lastErr = err
+ pg := ep.tty.ForegroundProcessGroup()
+ if pg == nil {
+ // No foreground process group has been set. Signal the
+ // original thread group.
+ log.Warningf("No foreground process group for container %q and PID %d. Sending signal directly to PID %d.", cid, pid, pid)
+ return ep.tg.SendSignal(&arch.SignalInfo{Signo: signo})
+ }
+ // Send the signal to all processes in the process group.
+ var lastErr error
+ for _, tg := range l.k.TaskSet().Root.ThreadGroups() {
+ if tg.ProcessGroup() != pg {
+ continue
+ }
+ if err := tg.SendSignal(&arch.SignalInfo{Signo: signo}); err != nil {
+ lastErr = err
+ }
+ }
+ return lastErr
+ case DeliverToAllProcesses:
+ if !ok {
+ return fmt.Errorf("failed to signal all processes in container %q PID %d: no such PID", cid, pid)
}
- }
- return lastErr
-}
-
-// signalContainer sends a signal to the root container process, or to all
-// processes in the container if all is true.
-func (l *Loader) signalContainer(cid string, signo int32, all bool) error {
- si := arch.SignalInfo{Signo: signo}
-
- l.mu.Lock()
- defer l.mu.Unlock()
-
- eid := execID{cid: cid}
- ep, ok := l.processes[eid]
- if !ok {
- return fmt.Errorf("failed to signal container %q: no such container", cid)
- }
-
- if !all {
- return ep.tg.SendSignal(&si)
- }
- // 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()
- if err := l.k.SendContainerSignal(cid, &si); err != nil {
+ // 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()
+ if err := l.k.SendContainerSignal(cid, &arch.SignalInfo{Signo: signo}); err != nil {
+ l.k.Unpause()
+ return err
+ }
l.k.Unpause()
- return err
- }
- l.k.Unpause()
- // If killing all processes, wait for them to exit.
- if all && linux.Signal(signo) == linux.SIGKILL {
- for _, t := range l.k.TaskSet().Root.Tasks() {
- if t.ContainerID() == cid {
- t.ThreadGroup().WaitExited()
+ // If SIGKILLing all processes, wait for them to exit.
+ if linux.Signal(signo) == linux.SIGKILL {
+ for _, t := range l.k.TaskSet().Root.Tasks() {
+ if t.ContainerID() == cid {
+ t.ThreadGroup().WaitExited()
+ }
}
}
+ return nil
+ default:
+ panic(fmt.Sprintf("unknown signal signal delivery mode %v", mode))
}
- return nil
}
diff --git a/runsc/container/BUILD b/runsc/container/BUILD
index 60f1d3033..f4c6f1525 100644
--- a/runsc/container/BUILD
+++ b/runsc/container/BUILD
@@ -30,6 +30,7 @@ go_test(
name = "container_test",
size = "medium",
srcs = [
+ "console_test.go",
"container_test.go",
"fs_test.go",
"multi_container_test.go",
diff --git a/runsc/container/console_test.go b/runsc/container/console_test.go
new file mode 100644
index 000000000..82adcbb7d
--- /dev/null
+++ b/runsc/container/console_test.go
@@ -0,0 +1,452 @@
+// Copyright 2018 Google Inc.
+//
+// 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 container
+
+import (
+ "bytes"
+ "fmt"
+ "io"
+ "os"
+ "path/filepath"
+ "sync"
+ "syscall"
+ "testing"
+ "time"
+
+ "github.com/kr/pty"
+ "golang.org/x/sys/unix"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/control"
+ "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.googlesource.com/gvisor/pkg/urpc"
+ "gvisor.googlesource.com/gvisor/runsc/test/testutil"
+)
+
+// createConsoleSocket creates a socket that will receive a console fd from the
+// sandbox. If no error occurs, it returns the server socket and a cleanup
+// function.
+func createConsoleSocket(socketPath string) (*unet.ServerSocket, func() error, error) {
+ cwd, err := os.Getwd()
+ if err != nil {
+ return nil, nil, fmt.Errorf("error getting cwd: %v", err)
+ }
+ // We use a relative path to avoid overflowing the unix path length
+ // limit (108 chars).
+ socketRelPath, err := filepath.Rel(cwd, socketPath)
+ if err != nil {
+ return nil, nil, fmt.Errorf("error getting relative path for %q from cwd %q: %v", socketPath, cwd, err)
+ }
+ if len(socketRelPath) > len(socketPath) {
+ socketRelPath = socketPath
+ }
+ srv, err := unet.BindAndListen(socketRelPath, false)
+ if err != nil {
+ return nil, nil, fmt.Errorf("error binding and listening to socket %q: %v", socketPath, err)
+ }
+
+ cleanup := func() error {
+ if err := srv.Close(); err != nil {
+ return fmt.Errorf("error closing socket %q: %v", socketRelPath, err)
+ }
+ if err := os.Remove(socketPath); err != nil {
+ return fmt.Errorf("error removing socket %q: %v", socketRelPath, err)
+ }
+ return nil
+ }
+
+ return srv, cleanup, nil
+}
+
+// receiveConsolePTY accepts a connection on the server socket and reads fds.
+// It fails if more than one FD is received, or if the FD is not a PTY. It
+// returns the PTY master file.
+func receiveConsolePTY(srv *unet.ServerSocket) (*os.File, error) {
+ sock, err := srv.Accept()
+ if err != nil {
+ return nil, fmt.Errorf("error accepting socket connection: %v", err)
+ }
+
+ // Allow 3 fds to be received. We only expect 1.
+ r := sock.Reader(true /* blocking */)
+ r.EnableFDs(1)
+
+ // The socket is closed right after sending the FD, so EOF is
+ // an allowed error.
+ b := [][]byte{{}}
+ if _, err := r.ReadVec(b); err != nil && err != io.EOF {
+ return nil, fmt.Errorf("error reading from socket connection: %v", err)
+ }
+
+ // We should have gotten a control message.
+ fds, err := r.ExtractFDs()
+ if err != nil {
+ return nil, fmt.Errorf("error extracting fds from socket connection: %v", err)
+ }
+ if len(fds) != 1 {
+ return nil, fmt.Errorf("got %d fds from socket, wanted 1", len(fds))
+ }
+
+ // Verify that the fd is a terminal.
+ if _, err := unix.IoctlGetTermios(fds[0], unix.TCGETS); err != nil {
+ return nil, fmt.Errorf("fd is not a terminal (ioctl TGGETS got %v)", err)
+ }
+
+ return os.NewFile(uintptr(fds[0]), "pty_master"), nil
+}
+
+// Test that an pty FD is sent over the console socket if one is provided.
+func TestConsoleSocket(t *testing.T) {
+ for _, conf := range configs(all...) {
+ t.Logf("Running test with conf: %+v", conf)
+ spec := testutil.NewSpecWithArgs("true")
+ rootDir, bundleDir, err := testutil.SetupContainer(spec, conf)
+ if err != nil {
+ t.Fatalf("error setting up container: %v", err)
+ }
+ defer os.RemoveAll(rootDir)
+ defer os.RemoveAll(bundleDir)
+
+ socketPath := filepath.Join(bundleDir, "socket")
+ srv, cleanup, err := createConsoleSocket(socketPath)
+ if err != nil {
+ t.Fatalf("error creating socket at %q: %v", socketPath, err)
+ }
+ defer cleanup()
+
+ // Create the container and pass the socket name.
+ id := testutil.UniqueContainerID()
+ c, err := Create(id, spec, conf, bundleDir, socketPath, "", "")
+ if err != nil {
+ t.Fatalf("error creating container: %v", err)
+ }
+ defer c.Destroy()
+
+ // Make sure we get a console PTY.
+ ptyMaster, err := receiveConsolePTY(srv)
+ if err != nil {
+ t.Fatalf("error receiving console FD: %v", err)
+ }
+ ptyMaster.Close()
+ }
+}
+
+// Test that job control signals work on a console created with "exec -ti".
+func TestJobControlSignalExec(t *testing.T) {
+ spec := testutil.NewSpecWithArgs("/bin/sleep", "10000")
+ conf := testutil.TestConfig()
+
+ rootDir, bundleDir, err := testutil.SetupContainer(spec, conf)
+ if err != nil {
+ t.Fatalf("error setting up container: %v", err)
+ }
+ defer os.RemoveAll(rootDir)
+ defer os.RemoveAll(bundleDir)
+
+ // Create and start the container.
+ c, err := Create(testutil.UniqueContainerID(), spec, conf, bundleDir, "", "", "")
+ if err != nil {
+ t.Fatalf("error creating container: %v", err)
+ }
+ defer c.Destroy()
+ if err := c.Start(conf); err != nil {
+ t.Fatalf("error starting container: %v", err)
+ }
+
+ // Create a pty master/slave. The slave will be passed to the exec
+ // process.
+ ptyMaster, ptySlave, err := pty.Open()
+ if err != nil {
+ t.Fatalf("error opening pty: %v", err)
+ }
+ defer ptyMaster.Close()
+ defer ptySlave.Close()
+
+ // Exec bash and attach a terminal.
+ args := &control.ExecArgs{
+ Filename: "/bin/bash",
+ // Don't let bash execute from profile or rc files, otherwise
+ // our PID counts get messed up.
+ Argv: []string{"/bin/bash", "--noprofile", "--norc"},
+ // Pass the pty slave as FD 0, 1, and 2.
+ FilePayload: urpc.FilePayload{
+ Files: []*os.File{ptySlave, ptySlave, ptySlave},
+ },
+ StdioIsPty: true,
+ }
+
+ pid, err := c.Execute(args)
+ if err != nil {
+ t.Fatalf("error executing: %v", err)
+ }
+ if pid != 2 {
+ t.Fatalf("exec got pid %d, wanted %d", pid, 2)
+ }
+
+ // Make sure all the processes are running.
+ expectedPL := []*control.Process{
+ // Root container process.
+ {PID: 1, Cmd: "sleep"},
+ // Bash from exec process.
+ {PID: 2, Cmd: "bash"},
+ }
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Error(err)
+ }
+
+ // Execute sleep.
+ ptyMaster.Write([]byte("sleep 100\n"))
+
+ // Wait for it to start. Sleep's PPID is bash's PID.
+ expectedPL = append(expectedPL, &control.Process{PID: 3, PPID: 2, Cmd: "sleep"})
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Error(err)
+ }
+
+ // Send a SIGTERM to the foreground process for the exec PID. Note that
+ // although we pass in the PID of "bash", it should actually terminate
+ // "sleep", since that is the foreground process.
+ if err := c.Sandbox.SignalProcess(c.ID, pid, syscall.SIGTERM, true /* fgProcess */); err != nil {
+ t.Fatalf("error signaling container: %v", err)
+ }
+
+ // Sleep process should be gone.
+ expectedPL = expectedPL[:len(expectedPL)-1]
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Error(err)
+ }
+
+ // Sleep is dead, but it may take more time for bash to notice and
+ // change the foreground process back to itself. We know it is done
+ // when bash writes "Terminated" to the pty.
+ if err := testutil.WaitUntilRead(ptyMaster, "Terminated", nil, 5*time.Second); err != nil {
+ t.Fatalf("bash did not take over pty: %v", err)
+ }
+
+ // Send a SIGKILL to the foreground process again. This time "bash"
+ // should be killed. We use SIGKILL instead of SIGTERM or SIGINT
+ // because bash ignores those.
+ if err := c.Sandbox.SignalProcess(c.ID, pid, syscall.SIGKILL, true /* fgProcess */); err != nil {
+ t.Fatalf("error signaling container: %v", err)
+ }
+ expectedPL = expectedPL[:1]
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Error(err)
+ }
+
+ // Make sure the process indicates it was killed by a SIGKILL.
+ ws, err := c.WaitPID(pid, true)
+ if err != nil {
+ t.Errorf("waiting on container failed: %v", err)
+ }
+ if !ws.Signaled() {
+ t.Error("ws.Signaled() got false, want true")
+ }
+ if got, want := ws.Signal(), syscall.SIGKILL; got != want {
+ t.Errorf("ws.Signal() got %v, want %v", got, want)
+ }
+}
+
+// Test that job control signals work on a console created with "run -ti".
+func TestJobControlSignalRootContainer(t *testing.T) {
+ conf := testutil.TestConfig()
+ // Don't let bash execute from profile or rc files, otherwise our PID
+ // counts get messed up.
+ spec := testutil.NewSpecWithArgs("/bin/bash", "--noprofile", "--norc")
+ spec.Process.Terminal = true
+
+ rootDir, bundleDir, err := testutil.SetupContainer(spec, conf)
+ if err != nil {
+ t.Fatalf("error setting up container: %v", err)
+ }
+ defer os.RemoveAll(rootDir)
+ defer os.RemoveAll(bundleDir)
+
+ socketPath := filepath.Join(bundleDir, "socket")
+ srv, cleanup, err := createConsoleSocket(socketPath)
+ if err != nil {
+ t.Fatalf("error creating socket at %q: %v", socketPath, err)
+ }
+ defer cleanup()
+
+ // Create the container and pass the socket name.
+ id := testutil.UniqueContainerID()
+ c, err := Create(id, spec, conf, bundleDir, socketPath, "", "")
+ if err != nil {
+ t.Fatalf("error creating container: %v", err)
+ }
+ defer c.Destroy()
+
+ // Get the PTY master.
+ ptyMaster, err := receiveConsolePTY(srv)
+ if err != nil {
+ t.Fatalf("error receiving console FD: %v", err)
+ }
+ defer ptyMaster.Close()
+
+ // Bash output as well as sandbox output will be written to the PTY
+ // file. Writes after a certain point will block unless we drain the
+ // PTY, so we must continually copy from it.
+ //
+ // We log the output to stdout for debugabilitly, and also to a buffer,
+ // since we wait on particular output from bash below. We use a custom
+ // blockingBuffer which is thread-safe and also blocks on Read calls,
+ // which makes this a suitable Reader for WaitUntilRead.
+ ptyBuf := newBlockingBuffer()
+ tee := io.TeeReader(ptyMaster, ptyBuf)
+ go io.Copy(os.Stdout, tee)
+
+ // Start the container.
+ if err := c.Start(conf); err != nil {
+ t.Fatalf("error starting container: %v", err)
+ }
+
+ // Start waiting for the container to exit in a goroutine. We do this
+ // very early, otherwise it might exit before we have a chance to call
+ // Wait.
+ var (
+ ws syscall.WaitStatus
+ wg sync.WaitGroup
+ )
+ wg.Add(1)
+ go func() {
+ var err error
+ ws, err = c.Wait()
+ if err != nil {
+ t.Errorf("error waiting on container: %v", err)
+ }
+ wg.Done()
+ }()
+
+ // Wait for bash to start.
+ expectedPL := []*control.Process{
+ {PID: 1, Cmd: "bash"},
+ }
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Fatal(err)
+ }
+
+ // Execute sleep via the terminal.
+ ptyMaster.Write([]byte("sleep 100\n"))
+
+ // Wait for sleep to start.
+ expectedPL = append(expectedPL, &control.Process{PID: 2, PPID: 1, Cmd: "sleep"})
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Fatal(err)
+ }
+
+ // Reset the pty buffer, so there is less output for us to scan later.
+ ptyBuf.Reset()
+
+ // Send a SIGTERM to the foreground process. We pass PID=0, indicating
+ // that the root process should be killed. However, by setting
+ // fgProcess=true, the signal should actually be sent to sleep.
+ if err := c.Sandbox.SignalProcess(c.ID, 0 /* PID */, syscall.SIGTERM, true /* fgProcess */); err != nil {
+ t.Fatalf("error signaling container: %v", err)
+ }
+
+ // Sleep process should be gone.
+ expectedPL = expectedPL[:len(expectedPL)-1]
+ if err := waitForProcessList(c, expectedPL); err != nil {
+ t.Error(err)
+ }
+
+ // Sleep is dead, but it may take more time for bash to notice and
+ // change the foreground process back to itself. We know it is done
+ // when bash writes "Terminated" to the pty.
+ if err := testutil.WaitUntilRead(ptyBuf, "Terminated", nil, 5*time.Second); err != nil {
+ t.Fatalf("bash did not take over pty: %v", err)
+ }
+
+ // Send a SIGKILL to the foreground process again. This time "bash"
+ // should be killed. We use SIGKILL instead of SIGTERM or SIGINT
+ // because bash ignores those.
+ if err := c.Sandbox.SignalProcess(c.ID, 0 /* PID */, syscall.SIGKILL, true /* fgProcess */); err != nil {
+ t.Fatalf("error signaling container: %v", err)
+ }
+
+ // Wait for the sandbox to exit. It should exit with a SIGKILL status.
+ wg.Wait()
+ if !ws.Signaled() {
+ t.Error("ws.Signaled() got false, want true")
+ }
+ if got, want := ws.Signal(), syscall.SIGKILL; got != want {
+ t.Errorf("ws.Signal() got %v, want %v", got, want)
+ }
+}
+
+// blockingBuffer is a thread-safe buffer that blocks when reading if the
+// buffer is empty. It implements io.ReadWriter.
+type blockingBuffer struct {
+ // A send to readCh indicates that a previously empty buffer now has
+ // data for reading.
+ readCh chan struct{}
+
+ // mu protects buf.
+ mu sync.Mutex
+ buf bytes.Buffer
+}
+
+func newBlockingBuffer() *blockingBuffer {
+ return &blockingBuffer{
+ readCh: make(chan struct{}, 1),
+ }
+}
+
+// Write implements Writer.Write.
+func (bb *blockingBuffer) Write(p []byte) (int, error) {
+ bb.mu.Lock()
+ defer bb.mu.Unlock()
+ l := bb.buf.Len()
+ n, err := bb.buf.Write(p)
+ if l == 0 && n > 0 {
+ // New data!
+ bb.readCh <- struct{}{}
+ }
+ return n, err
+}
+
+// Read implements Reader.Read. It will block until data is available.
+func (bb *blockingBuffer) Read(p []byte) (int, error) {
+ for {
+ bb.mu.Lock()
+ n, err := bb.buf.Read(p)
+ if n > 0 || err != io.EOF {
+ if bb.buf.Len() == 0 {
+ // Reset the readCh.
+ select {
+ case <-bb.readCh:
+ default:
+ }
+ }
+ bb.mu.Unlock()
+ return n, err
+ }
+ bb.mu.Unlock()
+
+ // Wait for new data.
+ <-bb.readCh
+ }
+}
+
+// Reset resets the buffer.
+func (bb *blockingBuffer) Reset() {
+ bb.mu.Lock()
+ defer bb.mu.Unlock()
+ bb.buf.Reset()
+ // Reset the readCh.
+ select {
+ case <-bb.readCh:
+ default:
+ }
+}
diff --git a/runsc/container/container_test.go b/runsc/container/container_test.go
index d9cd38c0a..e2bb7d8ec 100644
--- a/runsc/container/container_test.go
+++ b/runsc/container/container_test.go
@@ -17,7 +17,6 @@ package container
import (
"bytes"
"fmt"
- "io"
"io/ioutil"
"os"
"path"
@@ -31,15 +30,11 @@ import (
"time"
"github.com/cenkalti/backoff"
- "github.com/kr/pty"
specs "github.com/opencontainers/runtime-spec/specs-go"
- "golang.org/x/sys/unix"
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
"gvisor.googlesource.com/gvisor/pkg/log"
"gvisor.googlesource.com/gvisor/pkg/sentry/control"
"gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
- "gvisor.googlesource.com/gvisor/pkg/unet"
- "gvisor.googlesource.com/gvisor/pkg/urpc"
"gvisor.googlesource.com/gvisor/runsc/boot"
"gvisor.googlesource.com/gvisor/runsc/test/testutil"
)
@@ -1151,89 +1146,6 @@ func TestCapabilities(t *testing.T) {
}
}
-// Test that an tty FD is sent over the console socket if one is provided.
-func TestConsoleSocket(t *testing.T) {
- for _, conf := range configs(all...) {
- t.Logf("Running test with conf: %+v", conf)
- spec := testutil.NewSpecWithArgs("true")
- rootDir, bundleDir, err := testutil.SetupContainer(spec, conf)
- if err != nil {
- t.Fatalf("error setting up container: %v", err)
- }
- defer os.RemoveAll(rootDir)
- defer os.RemoveAll(bundleDir)
-
- // Create a named socket and start listening. We use a relative path
- // to avoid overflowing the unix path length limit (108 chars).
- socketPath := filepath.Join(bundleDir, "socket")
- cwd, err := os.Getwd()
- if err != nil {
- t.Fatalf("error getting cwd: %v", err)
- }
- socketRelPath, err := filepath.Rel(cwd, socketPath)
- if err != nil {
- t.Fatalf("error getting relative path for %q from cwd %q: %v", socketPath, cwd, err)
- }
- if len(socketRelPath) > len(socketPath) {
- socketRelPath = socketPath
- }
- srv, err := unet.BindAndListen(socketRelPath, false)
- if err != nil {
- t.Fatalf("error binding and listening to socket %q: %v", socketPath, err)
- }
- defer os.Remove(socketPath)
-
- // Create the container and pass the socket name.
- id := testutil.UniqueContainerID()
- c, err := Create(id, spec, conf, bundleDir, socketRelPath, "", "")
- if err != nil {
- t.Fatalf("error creating container: %v", err)
- }
- c.Destroy()
-
- // Open the othe end of the socket.
- sock, err := srv.Accept()
- if err != nil {
- t.Fatalf("error accepting socket connection: %v", err)
- }
-
- // Allow 3 fds to be received. We only expect 1.
- r := sock.Reader(true /* blocking */)
- r.EnableFDs(1)
-
- // The socket is closed right after sending the FD, so EOF is
- // an allowed error.
- b := [][]byte{{}}
- if _, err := r.ReadVec(b); err != nil && err != io.EOF {
- t.Fatalf("error reading from socket connection: %v", err)
- }
-
- // We should have gotten a control message.
- fds, err := r.ExtractFDs()
- if err != nil {
- t.Fatalf("error extracting fds from socket connection: %v", err)
- }
- if len(fds) != 1 {
- t.Fatalf("got %d fds from socket, wanted 1", len(fds))
- }
-
- // Verify that the fd is a terminal.
- if _, err := unix.IoctlGetTermios(fds[0], unix.TCGETS); err != nil {
- t.Errorf("fd is not a terminal (ioctl TGGETS got %v)", err)
- }
-
- // Shut it down.
- if err := c.Destroy(); err != nil {
- t.Fatalf("error destroying container: %v", err)
- }
-
- // Close socket.
- if err := srv.Close(); err != nil {
- t.Fatalf("error destroying container: %v", err)
- }
- }
-}
-
// TestRunNonRoot checks that sandbox can be configured when running as
// non-privileged user.
func TestRunNonRoot(t *testing.T) {
@@ -1626,121 +1538,6 @@ func TestRootNotMount(t *testing.T) {
}
}
-func TestJobControlSignalExec(t *testing.T) {
- spec := testutil.NewSpecWithArgs("/bin/sleep", "10000")
- conf := testutil.TestConfig()
-
- rootDir, bundleDir, err := testutil.SetupContainer(spec, conf)
- if err != nil {
- t.Fatalf("error setting up container: %v", err)
- }
- defer os.RemoveAll(rootDir)
- defer os.RemoveAll(bundleDir)
-
- // Create and start the container.
- c, err := Create(testutil.UniqueContainerID(), spec, conf, bundleDir, "", "", "")
- if err != nil {
- t.Fatalf("error creating container: %v", err)
- }
- defer c.Destroy()
- if err := c.Start(conf); err != nil {
- t.Fatalf("error starting container: %v", err)
- }
-
- // Create a pty master/slave. The slave will be passed to the exec
- // process.
- ptyMaster, ptySlave, err := pty.Open()
- if err != nil {
- t.Fatalf("error opening pty: %v", err)
- }
- defer ptyMaster.Close()
- defer ptySlave.Close()
-
- // Exec bash and attach a terminal.
- args := &control.ExecArgs{
- Filename: "/bin/bash",
- // Don't let bash execute from profile or rc files, otherwise
- // our PID counts get messed up.
- Argv: []string{"/bin/bash", "--noprofile", "--norc"},
- // Pass the pty slave as FD 0, 1, and 2.
- FilePayload: urpc.FilePayload{
- Files: []*os.File{ptySlave, ptySlave, ptySlave},
- },
- StdioIsPty: true,
- }
-
- pid, err := c.Execute(args)
- if err != nil {
- t.Fatalf("error executing: %v", err)
- }
- if pid != 2 {
- t.Fatalf("exec got pid %d, wanted %d", pid, 2)
- }
-
- // Make sure all the processes are running.
- expectedPL := []*control.Process{
- // Root container process.
- {PID: 1, Cmd: "sleep"},
- // Bash from exec process.
- {PID: 2, Cmd: "bash"},
- }
- if err := waitForProcessList(c, expectedPL); err != nil {
- t.Error(err)
- }
-
- // Execute sleep.
- ptyMaster.Write([]byte("sleep 100\n"))
-
- // Wait for it to start. Sleep's PPID is bash's PID.
- expectedPL = append(expectedPL, &control.Process{PID: 3, PPID: 2, Cmd: "sleep"})
- if err := waitForProcessList(c, expectedPL); err != nil {
- t.Error(err)
- }
-
- // Send a SIGTERM to the foreground process for the exec PID. Note that
- // although we pass in the PID of "bash", it should actually terminate
- // "sleep", since that is the foreground process.
- if err := c.Sandbox.SignalProcess(c.ID, pid, syscall.SIGTERM, true /* fgProcess */); err != nil {
- t.Fatalf("error signaling container: %v", err)
- }
-
- // Sleep process should be gone.
- expectedPL = expectedPL[:len(expectedPL)-1]
- if err := waitForProcessList(c, expectedPL); err != nil {
- t.Error(err)
- }
-
- // Sleep is dead, but it may take more time for bash to notice and
- // change the foreground process back to itself. We know it is done
- // when bash writes "Terminated" to the pty.
- if err := testutil.WaitUntilRead(ptyMaster, "Terminated", nil, 5*time.Second); err != nil {
- t.Fatalf("bash did not take over pty: %v", err)
- }
-
- // Send a SIGKILL to the foreground process again. This time "bash"
- // should be killed. We use SIGKILL instead of SIGTERM or SIGINT
- // because bash ignores those.
- if err := c.Sandbox.SignalProcess(c.ID, pid, syscall.SIGKILL, true /* fgProcess */); err != nil {
- t.Fatalf("error signaling container: %v", err)
- }
- expectedPL = expectedPL[:1]
- if err := waitForProcessList(c, expectedPL); err != nil {
- t.Error(err)
- }
-
- // Make sure the process indicates it was killed by a SIGKILL.
- ws, err := c.WaitPID(pid, true)
- if err != nil {
- t.Errorf("waiting on container failed: %v", err)
- }
- if !ws.Signaled() {
- t.Error("ws.Signaled() got false, want true")
- }
- if got, want := ws.Signal(), syscall.SIGKILL; got != want {
- t.Errorf("ws.Signal() got %v, want %v", got, want)
- }
-}
-
func TestUserLog(t *testing.T) {
app, err := testutil.FindFile("runsc/container/test_app")
if err != nil {
diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go
index 713b326a6..6dc8cf7f0 100644
--- a/runsc/sandbox/sandbox.go
+++ b/runsc/sandbox/sandbox.go
@@ -696,10 +696,15 @@ func (s *Sandbox) SignalContainer(cid string, sig syscall.Signal, all bool) erro
}
defer conn.Close()
+ mode := boot.DeliverToProcess
+ if all {
+ mode = boot.DeliverToAllProcesses
+ }
+
args := boot.SignalArgs{
CID: cid,
Signo: int32(sig),
- All: all,
+ Mode: mode,
}
if err := conn.Call(boot.ContainerSignal, &args, nil); err != nil {
return fmt.Errorf("err signaling container %q: %v", cid, err)
@@ -719,13 +724,18 @@ func (s *Sandbox) SignalProcess(cid string, pid int32, sig syscall.Signal, fgPro
}
defer conn.Close()
- args := boot.SignalProcessArgs{
- CID: cid,
- Signo: int32(sig),
- PID: pid,
- SendToForegroundProcess: fgProcess,
+ mode := boot.DeliverToProcess
+ if fgProcess {
+ mode = boot.DeliverToForegroundProcessGroup
+ }
+
+ args := boot.SignalArgs{
+ CID: cid,
+ Signo: int32(sig),
+ PID: pid,
+ Mode: mode,
}
- if err := conn.Call(boot.ContainerSignalProcess, &args, nil); err != nil {
+ if err := conn.Call(boot.ContainerSignal, &args, nil); err != nil {
return fmt.Errorf("err signaling container %q PID %d: %v", cid, pid, err)
}
return nil
diff --git a/runsc/test/integration/exec_test.go b/runsc/test/integration/exec_test.go
index d08140ad3..3cac674d0 100644
--- a/runsc/test/integration/exec_test.go
+++ b/runsc/test/integration/exec_test.go
@@ -66,7 +66,7 @@ func TestExecJobControl(t *testing.T) {
if err := testutil.Pull("alpine"); err != nil {
t.Fatalf("docker pull failed: %v", err)
}
- d := testutil.MakeDocker("exec-test")
+ d := testutil.MakeDocker("exec-job-control-test")
// Start the container.
if err := d.Run("alpine", "sleep", "1000"); err != nil {
diff --git a/runsc/test/integration/integration_test.go b/runsc/test/integration/integration_test.go
index b7d07309d..536bb17e0 100644
--- a/runsc/test/integration/integration_test.go
+++ b/runsc/test/integration/integration_test.go
@@ -28,6 +28,7 @@ import (
"os"
"strconv"
"strings"
+ "syscall"
"testing"
"time"
@@ -231,6 +232,53 @@ func TestNumCPU(t *testing.T) {
}
}
+// TestJobControl tests that job control characters are handled properly.
+func TestJobControl(t *testing.T) {
+ if err := testutil.Pull("alpine"); err != nil {
+ t.Fatalf("docker pull failed: %v", err)
+ }
+ d := testutil.MakeDocker("job-control-test")
+
+ // Start the container with an attached PTY.
+ _, ptmx, err := d.RunWithPty("alpine", "sh")
+ if err != nil {
+ t.Fatalf("docker run failed: %v", err)
+ }
+ defer ptmx.Close()
+ defer d.CleanUp()
+
+ // Call "sleep 100" in the shell.
+ if _, err := ptmx.Write([]byte("sleep 100\n")); err != nil {
+ t.Fatalf("error writing to pty: %v", err)
+ }
+
+ // Give shell a few seconds to start executing the sleep.
+ time.Sleep(2 * time.Second)
+
+ // Send a ^C to the pty, which should kill sleep, but not the shell.
+ // \x03 is ASCII "end of text", which is the same as ^C.
+ if _, err := ptmx.Write([]byte{'\x03'}); err != nil {
+ t.Fatalf("error writing to pty: %v", err)
+ }
+
+ // The shell should still be alive at this point. Sleep should have
+ // exited with code 2+128=130. We'll exit with 10 plus that number, so
+ // that we can be sure that the shell did not get signalled.
+ if _, err := ptmx.Write([]byte("exit $(expr $? + 10)\n")); err != nil {
+ t.Fatalf("error writing to pty: %v", err)
+ }
+
+ // Wait for the container to exit.
+ got, err := d.Wait(5 * time.Second)
+ if err != nil {
+ t.Fatalf("error getting exit code: %v", err)
+ }
+ // Container should exit with code 10+130=140.
+ if want := syscall.WaitStatus(140); got != want {
+ t.Errorf("container exited with code %d want %d", got, want)
+ }
+}
+
func TestMain(m *testing.M) {
testutil.EnsureSupportedDockerVersion()
os.Exit(m.Run())
diff --git a/runsc/test/testutil/docker.go b/runsc/test/testutil/docker.go
index 8a51d3eed..4e48817cf 100644
--- a/runsc/test/testutil/docker.go
+++ b/runsc/test/testutil/docker.go
@@ -25,6 +25,7 @@ import (
"regexp"
"strconv"
"strings"
+ "syscall"
"time"
"github.com/kr/pty"
@@ -198,6 +199,13 @@ func (d *Docker) Run(args ...string) error {
return err
}
+// RunWithPty is like Run but with an attached pty.
+func (d *Docker) RunWithPty(args ...string) (*exec.Cmd, *os.File, error) {
+ a := []string{"run", "--runtime", d.Runtime, "--name", d.Name, "-it"}
+ a = append(a, args...)
+ return doWithPty(a...)
+}
+
// RunFg calls 'docker run' with the arguments provided in the foreground. It
// blocks until the container exits and returns the output.
func (d *Docker) RunFg(args ...string) (string, error) {
@@ -307,6 +315,37 @@ func (d *Docker) ID() (string, error) {
return strings.TrimSpace(string(out)), nil
}
+// Wait waits for container to exit, up to the given timeout. Returns error if
+// wait fails or timeout is hit. Returns the application return code otherwise.
+// Note that the application may have failed even if err == nil, always check
+// the exit code.
+func (d *Docker) Wait(timeout time.Duration) (syscall.WaitStatus, error) {
+ timeoutChan := time.After(timeout)
+ waitChan := make(chan (syscall.WaitStatus))
+ errChan := make(chan (error))
+
+ go func() {
+ out, err := do("wait", d.Name)
+ if err != nil {
+ errChan <- fmt.Errorf("error waiting for container %q: %v", d.Name, err)
+ }
+ exit, err := strconv.Atoi(strings.TrimSuffix(string(out), "\n"))
+ if err != nil {
+ errChan <- fmt.Errorf("error parsing exit code %q: %v", out, err)
+ }
+ waitChan <- syscall.WaitStatus(uint32(exit))
+ }()
+
+ select {
+ case ws := <-waitChan:
+ return ws, nil
+ case err := <-errChan:
+ return syscall.WaitStatus(1), err
+ case <-timeoutChan:
+ return syscall.WaitStatus(1), fmt.Errorf("timeout waiting for container %q", d.Name)
+ }
+}
+
// WaitForOutput calls 'docker logs' to retrieve containers output and searches
// for the given pattern.
func (d *Docker) WaitForOutput(pattern string, timeout time.Duration) (string, error) {