summaryrefslogtreecommitdiffhomepage
path: root/runsc/boot/loader.go
diff options
context:
space:
mode:
Diffstat (limited to 'runsc/boot/loader.go')
-rw-r--r--runsc/boot/loader.go422
1 files changed, 301 insertions, 121 deletions
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index 9f0d5d7af..002479612 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -26,16 +26,19 @@ import (
specs "github.com/opencontainers/runtime-spec/specs-go"
"golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/abi"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/cpuid"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/memutil"
"gvisor.dev/gvisor/pkg/rand"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/control"
+ "gvisor.dev/gvisor/pkg/sentry/fdimport"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/host"
+ "gvisor.dev/gvisor/pkg/sentry/fs/user"
+ hostvfs2 "gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
"gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -46,9 +49,11 @@ import (
"gvisor.dev/gvisor/pkg/sentry/syscalls/linux/vfs2"
"gvisor.dev/gvisor/pkg/sentry/time"
"gvisor.dev/gvisor/pkg/sentry/usage"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sentry/watchdog"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/tcpip"
+ "gvisor.dev/gvisor/pkg/tcpip/link/loopback"
"gvisor.dev/gvisor/pkg/tcpip/link/sniffer"
"gvisor.dev/gvisor/pkg/tcpip/network/arp"
"gvisor.dev/gvisor/pkg/tcpip/network/ipv4"
@@ -60,6 +65,7 @@ import (
"gvisor.dev/gvisor/pkg/tcpip/transport/udp"
"gvisor.dev/gvisor/runsc/boot/filter"
_ "gvisor.dev/gvisor/runsc/boot/platforms" // register all platforms.
+ "gvisor.dev/gvisor/runsc/boot/pprof"
"gvisor.dev/gvisor/runsc/specutils"
// Include supported socket providers.
@@ -137,6 +143,9 @@ type execProcess struct {
// tty will be nil if the process is not attached to a terminal.
tty *host.TTYFileOperations
+ // tty will be nil if the process is not attached to a terminal.
+ ttyVFS2 *hostvfs2.TTYFileDescription
+
// pidnsPath is the pid namespace path in spec
pidnsPath string
}
@@ -154,13 +163,17 @@ type Args struct {
Spec *specs.Spec
// Conf is the system configuration.
Conf *Config
- // ControllerFD is the FD to the URPC controller.
+ // ControllerFD is the FD to the URPC controller. The Loader takes ownership
+ // of this FD and may close it at any time.
ControllerFD int
- // Device is an optional argument that is passed to the platform.
+ // Device is an optional argument that is passed to the platform. The Loader
+ // takes ownership of this file and may close it at any time.
Device *os.File
- // GoferFDs is an array of FDs used to connect with the Gofer.
+ // GoferFDs is an array of FDs used to connect with the Gofer. The Loader
+ // takes ownership of these FDs and may close them at any time.
GoferFDs []int
- // StdioFDs is the stdio for the application.
+ // StdioFDs is the stdio for the application. The Loader takes ownership of
+ // these FDs and may close them at any time.
StdioFDs []int
// Console is set to true if using TTY.
Console bool
@@ -173,6 +186,9 @@ type Args struct {
UserLogFD int
}
+// make sure stdioFDs are always the same on initial start and on restore
+const startingStdioFD = 64
+
// New initializes a new kernel loader configured by spec.
// New also handles setting up a kernel for restoring a container.
func New(args Args) (*Loader, error) {
@@ -186,11 +202,10 @@ func New(args Args) (*Loader, error) {
return nil, fmt.Errorf("setting up memory usage: %v", err)
}
+ // Is this a VFSv2 kernel?
if args.Conf.VFS2 {
- st, ok := kernel.LookupSyscallTable(abi.Linux, arch.Host)
- if ok {
- vfs2.Override(st.Table)
- }
+ kernel.VFS2Enabled = true
+ vfs2.Override()
}
// Create kernel and platform.
@@ -230,11 +245,8 @@ func New(args Args) (*Loader, error) {
return nil, fmt.Errorf("enabling strace: %v", err)
}
- // Create an empty network stack because the network namespace may be empty at
- // this point. Netns is configured before Run() is called. Netstack is
- // configured using a control uRPC message. Host network is configured inside
- // Run().
- networkStack, err := newEmptyNetworkStack(args.Conf, k, k)
+ // Create root network namespace/stack.
+ netns, err := newRootNetworkNamespace(args.Conf, k, k)
if err != nil {
return nil, fmt.Errorf("creating network: %v", err)
}
@@ -277,7 +289,7 @@ func New(args Args) (*Loader, error) {
FeatureSet: cpuid.HostFeatureSet(),
Timekeeper: tk,
RootUserNamespace: creds.UserNamespace,
- NetworkStack: networkStack,
+ RootNetworkNamespace: netns,
ApplicationCores: uint(args.NumCPU),
Vdso: vdso,
RootUTSNamespace: kernel.NewUTSNamespace(args.Spec.Hostname, args.Spec.Hostname, creds.UserNamespace),
@@ -320,6 +332,38 @@ func New(args Args) (*Loader, error) {
return nil, fmt.Errorf("creating pod mount hints: %v", err)
}
+ if kernel.VFS2Enabled {
+ // Set up host mount that will be used for imported fds.
+ hostFilesystem, err := hostvfs2.NewFilesystem(k.VFS())
+ if err != nil {
+ return nil, fmt.Errorf("failed to create hostfs filesystem: %v", err)
+ }
+ defer hostFilesystem.DecRef()
+ hostMount, err := k.VFS().NewDisconnectedMount(hostFilesystem, nil, &vfs.MountOptions{})
+ if err != nil {
+ return nil, fmt.Errorf("failed to create hostfs mount: %v", err)
+ }
+ k.SetHostMount(hostMount)
+ }
+
+ // Make host FDs stable between invocations. Host FDs must map to the exact
+ // same number when the sandbox is restored. Otherwise the wrong FD will be
+ // used.
+ var stdioFDs []int
+ newfd := startingStdioFD
+ for _, fd := range args.StdioFDs {
+ err := syscall.Dup3(fd, newfd, syscall.O_CLOEXEC)
+ if err != nil {
+ return nil, fmt.Errorf("dup3 of stdioFDs failed: %v", err)
+ }
+ stdioFDs = append(stdioFDs, newfd)
+ err = syscall.Close(fd)
+ if err != nil {
+ return nil, fmt.Errorf("close original stdioFDs failed: %v", err)
+ }
+ newfd++
+ }
+
eid := execID{cid: args.ID}
l := &Loader{
k: k,
@@ -328,7 +372,7 @@ func New(args Args) (*Loader, error) {
watchdog: dog,
spec: args.Spec,
goferFDs: args.GoferFDs,
- stdioFDs: args.StdioFDs,
+ stdioFDs: stdioFDs,
rootProcArgs: procArgs,
sandboxID: args.ID,
processes: map[execID]*execProcess{eid: {}},
@@ -368,11 +412,16 @@ func newProcess(id string, spec *specs.Spec, creds *auth.Credentials, k *kernel.
return kernel.CreateProcessArgs{}, fmt.Errorf("creating limits: %v", err)
}
+ wd := spec.Process.Cwd
+ if wd == "" {
+ wd = "/"
+ }
+
// Create the process arguments.
procArgs := kernel.CreateProcessArgs{
Argv: spec.Process.Args,
Envv: spec.Process.Env,
- WorkingDirectory: spec.Process.Cwd, // Defaults to '/' if empty.
+ WorkingDirectory: wd,
Credentials: creds,
Umask: 0022,
Limits: ls,
@@ -466,7 +515,7 @@ func (l *Loader) run() error {
// Delay host network configuration to this point because network namespace
// is configured after the loader is created and before Run() is called.
log.Debugf("Configuring host network")
- stack := l.k.NetworkStack().(*hostinet.Stack)
+ stack := l.k.RootNetworkNamespace().Stack().(*hostinet.Stack)
if err := stack.Configure(); err != nil {
return err
}
@@ -483,9 +532,11 @@ func (l *Loader) run() error {
// If we are restoring, we do not want to create a process.
// l.restore is set by the container manager when a restore call is made.
+ var ttyFile *host.TTYFileOperations
+ var ttyFileVFS2 *hostvfs2.TTYFileDescription
if !l.restore {
if l.conf.ProfileEnable {
- initializePProf()
+ pprof.Initialize()
}
// Finally done with all configuration. Setup filters before user code
@@ -497,13 +548,14 @@ func (l *Loader) run() error {
// Create the FD map, which will set stdin, stdout, and stderr. If console
// is true, then ioctl calls will be passed through to the host fd.
ctx := l.rootProcArgs.NewContext(l.k)
- fdTable, err := createFDTable(ctx, l.console, l.stdioFDs)
+ var err error
+
+ // CreateProcess takes a reference on FDMap if successful. We won't need
+ // ours either way.
+ l.rootProcArgs.FDTable, ttyFile, ttyFileVFS2, err = createFDTable(ctx, l.console, l.stdioFDs)
if err != nil {
return fmt.Errorf("importing fds: %v", err)
}
- // CreateProcess takes a reference on FDMap if successful. We won't need
- // ours either way.
- l.rootProcArgs.FDTable = fdTable
// Setup the root container file system.
l.startGoferMonitor(l.sandboxID, l.goferFDs)
@@ -517,7 +569,15 @@ func (l *Loader) run() error {
}
// Add the HOME enviroment variable if it is not already set.
- envv, err := maybeAddExecUserHome(ctx, l.rootProcArgs.MountNamespace, l.rootProcArgs.Credentials.RealKUID, l.rootProcArgs.Envv)
+ var envv []string
+ if kernel.VFS2Enabled {
+ envv, err = user.MaybeAddExecUserHomeVFS2(ctx, l.rootProcArgs.MountNamespaceVFS2,
+ l.rootProcArgs.Credentials.RealKUID, l.rootProcArgs.Envv)
+
+ } else {
+ envv, err = user.MaybeAddExecUserHome(ctx, l.rootProcArgs.MountNamespace,
+ l.rootProcArgs.Credentials.RealKUID, l.rootProcArgs.Envv)
+ }
if err != nil {
return err
}
@@ -538,14 +598,16 @@ func (l *Loader) run() error {
ep.pidnsPath = ns.Path
}
if l.console {
- ttyFile, _ := l.rootProcArgs.FDTable.Get(0)
- defer ttyFile.DecRef()
- ep.tty = ttyFile.FileOperations.(*host.TTYFileOperations)
-
- // Set the foreground process group on the TTY to the global
- // init process group, since that is what we are about to
- // start running.
- ep.tty.InitForegroundProcessGroup(ep.tg.ProcessGroup())
+ // Set the foreground process group on the TTY to the global init process
+ // group, since that is what we are about to start running.
+ switch {
+ case ttyFileVFS2 != nil:
+ ep.ttyVFS2 = ttyFileVFS2
+ ttyFileVFS2.InitForegroundProcessGroup(ep.tg.ProcessGroup())
+ case ttyFile != nil:
+ ep.tty = ttyFile
+ ttyFile.InitForegroundProcessGroup(ep.tg.ProcessGroup())
+ }
}
// Handle signals by forwarding them to the root container process
@@ -570,6 +632,19 @@ func (l *Loader) run() error {
}
})
+ // l.stdioFDs are derived from dup() in boot.New() and they are now dup()ed again
+ // either in createFDTable() during initial start or in descriptor.initAfterLoad()
+ // during restore, we can release l.stdioFDs now. VFS2 takes ownership of the
+ // passed FDs, so only close for VFS1.
+ if !kernel.VFS2Enabled {
+ for _, fd := range l.stdioFDs {
+ err := syscall.Close(fd)
+ if err != nil {
+ return fmt.Errorf("close dup()ed stdioFDs: %v", err)
+ }
+ }
+ }
+
log.Infof("Process should have started...")
l.watchdog.Start()
return l.k.Start()
@@ -653,7 +728,7 @@ func (l *Loader) startContainer(spec *specs.Spec, conf *Config, cid string, file
// Create the FD map, which will set stdin, stdout, and stderr.
ctx := procArgs.NewContext(l.k)
- fdTable, err := createFDTable(ctx, false, stdioFDs)
+ fdTable, _, _, err := createFDTable(ctx, false, stdioFDs)
if err != nil {
return fmt.Errorf("importing fds: %v", err)
}
@@ -738,14 +813,14 @@ func (l *Loader) destroyContainer(cid string) error {
l.mu.Lock()
defer l.mu.Unlock()
- _, _, started, err := l.threadGroupFromIDLocked(execID{cid: cid})
+ tg, err := l.tryThreadGroupFromIDLocked(execID{cid: cid})
if err != nil {
// Container doesn't exist.
return err
}
- // The container exists, has it been started?
- if started {
+ // The container exists, but has it been started?
+ if tg != nil {
if err := l.signalAllProcesses(cid, int32(linux.SIGKILL)); err != nil {
return fmt.Errorf("sending SIGKILL to all container processes: %v", err)
}
@@ -787,45 +862,65 @@ func (l *Loader) executeAsync(args *control.ExecArgs) (kernel.ThreadID, error) {
l.mu.Lock()
defer l.mu.Unlock()
- tg, _, started, err := l.threadGroupFromIDLocked(execID{cid: args.ContainerID})
+ tg, err := l.tryThreadGroupFromIDLocked(execID{cid: args.ContainerID})
if err != nil {
return 0, err
}
- if !started {
+ if tg == nil {
return 0, fmt.Errorf("container %q not started", args.ContainerID)
}
// Get the container MountNamespace from the Task.
- tg.Leader().WithMuLocked(func(t *kernel.Task) {
- // task.MountNamespace() does not take a ref, so we must do so
- // ourselves.
- args.MountNamespace = t.MountNamespace()
- args.MountNamespace.IncRef()
- })
- defer args.MountNamespace.DecRef()
+ if kernel.VFS2Enabled {
+ // task.MountNamespace() does not take a ref, so we must do so ourselves.
+ args.MountNamespaceVFS2 = tg.Leader().MountNamespaceVFS2()
+ args.MountNamespaceVFS2.IncRef()
+ } else {
+ tg.Leader().WithMuLocked(func(t *kernel.Task) {
+ // task.MountNamespace() does not take a ref, so we must do so ourselves.
+ args.MountNamespace = t.MountNamespace()
+ args.MountNamespace.IncRef()
+ })
+ }
- // Add the HOME enviroment varible if it is not already set.
- root := args.MountNamespace.Root()
- defer root.DecRef()
- ctx := fs.WithRoot(l.k.SupervisorContext(), root)
- envv, err := maybeAddExecUserHome(ctx, args.MountNamespace, args.KUID, args.Envv)
- if err != nil {
- return 0, err
+ // Add the HOME environment variable if it is not already set.
+ if kernel.VFS2Enabled {
+ defer args.MountNamespaceVFS2.DecRef()
+
+ root := args.MountNamespaceVFS2.Root()
+ defer root.DecRef()
+ ctx := vfs.WithRoot(l.k.SupervisorContext(), root)
+ envv, err := user.MaybeAddExecUserHomeVFS2(ctx, args.MountNamespaceVFS2, args.KUID, args.Envv)
+ if err != nil {
+ return 0, err
+ }
+ args.Envv = envv
+ } else {
+ defer args.MountNamespace.DecRef()
+
+ root := args.MountNamespace.Root()
+ defer root.DecRef()
+ ctx := fs.WithRoot(l.k.SupervisorContext(), root)
+ envv, err := user.MaybeAddExecUserHome(ctx, args.MountNamespace, args.KUID, args.Envv)
+ if err != nil {
+ return 0, err
+ }
+ args.Envv = envv
}
- args.Envv = envv
// Start the process.
proc := control.Proc{Kernel: l.k}
args.PIDNamespace = tg.PIDNamespace()
- newTG, tgid, ttyFile, err := control.ExecAsync(&proc, args)
+ newTG, tgid, ttyFile, ttyFileVFS2, err := control.ExecAsync(&proc, args)
if err != nil {
return 0, err
}
eid := execID{cid: args.ContainerID, pid: tgid}
l.processes[eid] = &execProcess{
- tg: newTG,
- tty: ttyFile,
+ tg: newTG,
+ tty: ttyFile,
+ ttyVFS2: ttyFileVFS2,
}
log.Debugf("updated processes: %v", l.processes)
@@ -836,7 +931,7 @@ func (l *Loader) executeAsync(args *control.ExecArgs) (kernel.ThreadID, error) {
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.
- tg, _, err := l.threadGroupFromID(execID{cid: cid})
+ tg, err := l.threadGroupFromID(execID{cid: cid})
if err != nil {
return fmt.Errorf("can't wait for container %q: %v", cid, err)
}
@@ -855,7 +950,7 @@ func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, waitStatus *uint32) e
// Try to find a process that was exec'd
eid := execID{cid: cid, pid: tgid}
- execTG, _, err := l.threadGroupFromID(eid)
+ execTG, err := l.threadGroupFromID(eid)
if err == nil {
ws := l.wait(execTG)
*waitStatus = ws
@@ -869,7 +964,7 @@ func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, waitStatus *uint32) e
// The caller may be waiting on a process not started directly via exec.
// In this case, find the process in the container's PID namespace.
- initTG, _, err := l.threadGroupFromID(execID{cid: cid})
+ initTG, err := l.threadGroupFromID(execID{cid: cid})
if err != nil {
return fmt.Errorf("waiting for PID %d: %v", tgid, err)
}
@@ -905,48 +1000,92 @@ func (l *Loader) WaitExit() kernel.ExitStatus {
return l.k.GlobalInit().ExitStatus()
}
-func newEmptyNetworkStack(conf *Config, clock tcpip.Clock, uniqueID stack.UniqueID) (inet.Stack, error) {
+func newRootNetworkNamespace(conf *Config, clock tcpip.Clock, uniqueID stack.UniqueID) (*inet.Namespace, error) {
+ // Create an empty network stack because the network namespace may be empty at
+ // this point. Netns is configured before Run() is called. Netstack is
+ // configured using a control uRPC message. Host network is configured inside
+ // Run().
switch conf.Network {
case NetworkHost:
- return hostinet.NewStack(), nil
+ // No network namespacing support for hostinet yet, hence creator is nil.
+ return inet.NewRootNamespace(hostinet.NewStack(), nil), nil
case NetworkNone, NetworkSandbox:
- // NetworkNone sets up loopback using netstack.
- netProtos := []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol(), arp.NewProtocol()}
- transProtos := []stack.TransportProtocol{tcp.NewProtocol(), udp.NewProtocol(), icmp.NewProtocol4()}
- s := netstack.Stack{stack.New(stack.Options{
- NetworkProtocols: netProtos,
- TransportProtocols: transProtos,
- Clock: clock,
- Stats: netstack.Metrics,
- HandleLocal: true,
- // Enable raw sockets for users with sufficient
- // privileges.
- RawFactory: raw.EndpointFactory{},
- UniqueID: uniqueID,
- })}
-
- // Enable SACK Recovery.
- if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcp.SACKEnabled(true)); err != nil {
- return nil, fmt.Errorf("failed to enable SACK: %v", err)
+ s, err := newEmptySandboxNetworkStack(clock, uniqueID)
+ if err != nil {
+ return nil, err
+ }
+ creator := &sandboxNetstackCreator{
+ clock: clock,
+ uniqueID: uniqueID,
}
+ return inet.NewRootNamespace(s, creator), nil
- // Set default TTLs as required by socket/netstack.
- s.Stack.SetNetworkProtocolOption(ipv4.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
- s.Stack.SetNetworkProtocolOption(ipv6.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
+ default:
+ panic(fmt.Sprintf("invalid network configuration: %v", conf.Network))
+ }
- // Enable Receive Buffer Auto-Tuning.
- if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcpip.ModerateReceiveBufferOption(true)); err != nil {
- return nil, fmt.Errorf("SetTransportProtocolOption failed: %v", err)
- }
+}
+
+func newEmptySandboxNetworkStack(clock tcpip.Clock, uniqueID stack.UniqueID) (inet.Stack, error) {
+ netProtos := []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol(), arp.NewProtocol()}
+ transProtos := []stack.TransportProtocol{tcp.NewProtocol(), udp.NewProtocol(), icmp.NewProtocol4()}
+ s := netstack.Stack{stack.New(stack.Options{
+ NetworkProtocols: netProtos,
+ TransportProtocols: transProtos,
+ Clock: clock,
+ Stats: netstack.Metrics,
+ HandleLocal: true,
+ // Enable raw sockets for users with sufficient
+ // privileges.
+ RawFactory: raw.EndpointFactory{},
+ UniqueID: uniqueID,
+ })}
- s.FillDefaultIPTables()
+ // Enable SACK Recovery.
+ if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcp.SACKEnabled(true)); err != nil {
+ return nil, fmt.Errorf("failed to enable SACK: %v", err)
+ }
- return &s, nil
+ // Set default TTLs as required by socket/netstack.
+ s.Stack.SetNetworkProtocolOption(ipv4.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
+ s.Stack.SetNetworkProtocolOption(ipv6.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
- default:
- panic(fmt.Sprintf("invalid network configuration: %v", conf.Network))
+ // Enable Receive Buffer Auto-Tuning.
+ if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcpip.ModerateReceiveBufferOption(true)); err != nil {
+ return nil, fmt.Errorf("SetTransportProtocolOption failed: %v", err)
+ }
+
+ s.FillIPTablesMetadata()
+
+ return &s, nil
+}
+
+// sandboxNetstackCreator implements kernel.NetworkStackCreator.
+//
+// +stateify savable
+type sandboxNetstackCreator struct {
+ clock tcpip.Clock
+ uniqueID stack.UniqueID
+}
+
+// CreateStack implements kernel.NetworkStackCreator.CreateStack.
+func (f *sandboxNetstackCreator) CreateStack() (inet.Stack, error) {
+ s, err := newEmptySandboxNetworkStack(f.clock, f.uniqueID)
+ if err != nil {
+ return nil, err
+ }
+
+ // Setup loopback.
+ n := &Network{Stack: s.(*netstack.Stack).Stack}
+ nicID := tcpip.NICID(f.uniqueID.UniqueID())
+ link := DefaultLoopbackLink
+ linkEP := loopback.New()
+ if err := n.createNICWithAddrs(nicID, link.Name, linkEP, link.Addresses); err != nil {
+ return nil, err
}
+
+ return s, nil
}
// signal sends a signal to one or more processes in a container. If PID is 0,
@@ -976,8 +1115,7 @@ func (l *Loader) signal(cid string, pid, signo int32, mode SignalDeliveryMode) e
return fmt.Errorf("PID (%d) cannot be set when signaling all processes", pid)
}
// Check that the container has actually started before signaling it.
- _, _, err := l.threadGroupFromID(execID{cid: cid})
- if err != nil {
+ if _, err := l.threadGroupFromID(execID{cid: cid}); err != nil {
return err
}
if err := l.signalAllProcesses(cid, signo); err != nil {
@@ -991,16 +1129,16 @@ func (l *Loader) signal(cid string, pid, signo int32, mode SignalDeliveryMode) e
}
func (l *Loader) signalProcess(cid string, tgid kernel.ThreadID, signo int32) error {
- execTG, _, err := l.threadGroupFromID(execID{cid: cid, pid: tgid})
+ execTG, err := l.threadGroupFromID(execID{cid: cid, pid: tgid})
if err == nil {
// Send signal directly to the identified process.
- return execTG.SendSignal(&arch.SignalInfo{Signo: signo})
+ return l.k.SendExternalSignalThreadGroup(execTG, &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.
- initTG, _, err := l.threadGroupFromID(execID{cid: cid})
+ initTG, err := l.threadGroupFromID(execID{cid: cid})
if err != nil {
return fmt.Errorf("no thread group found: %v", err)
}
@@ -1011,25 +1149,43 @@ func (l *Loader) signalProcess(cid string, tgid kernel.ThreadID, signo int32) er
if tg.Leader().ContainerID() != cid {
return fmt.Errorf("process %d is part of a different container: %q", tgid, tg.Leader().ContainerID())
}
- return tg.SendSignal(&arch.SignalInfo{Signo: signo})
+ return l.k.SendExternalSignalThreadGroup(tg, &arch.SignalInfo{Signo: signo})
}
+// signalForegrondProcessGroup looks up foreground process group from the TTY
+// for the given "tgid" inside container "cid", and send the signal to it.
func (l *Loader) signalForegrondProcessGroup(cid string, tgid kernel.ThreadID, signo int32) error {
- // Lookup foreground process group from the TTY for the given process,
- // and send the signal to it.
- tg, tty, err := l.threadGroupFromID(execID{cid: cid, pid: tgid})
+ l.mu.Lock()
+ tg, err := l.tryThreadGroupFromIDLocked(execID{cid: cid, pid: tgid})
if err != nil {
+ l.mu.Unlock()
return fmt.Errorf("no thread group found: %v", err)
}
- if tty == nil {
+ if tg == nil {
+ l.mu.Unlock()
+ return fmt.Errorf("container %q not started", cid)
+ }
+
+ tty, ttyVFS2, err := l.ttyFromIDLocked(execID{cid: cid, pid: tgid})
+ l.mu.Unlock()
+ if err != nil {
+ return fmt.Errorf("no thread group found: %v", err)
+ }
+
+ var pg *kernel.ProcessGroup
+ switch {
+ case ttyVFS2 != nil:
+ pg = ttyVFS2.ForegroundProcessGroup()
+ case tty != nil:
+ pg = tty.ForegroundProcessGroup()
+ default:
return fmt.Errorf("no TTY attached")
}
- pg := 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, tgid, tgid)
- return tg.SendSignal(&arch.SignalInfo{Signo: signo})
+ return l.k.SendExternalSignalThreadGroup(tg, &arch.SignalInfo{Signo: signo})
}
// Send the signal to all processes in the process group.
var lastErr error
@@ -1037,7 +1193,7 @@ func (l *Loader) signalForegrondProcessGroup(cid string, tgid kernel.ThreadID, s
if tg.ProcessGroup() != pg {
continue
}
- if err := tg.SendSignal(&arch.SignalInfo{Signo: signo}); err != nil {
+ if err := l.k.SendExternalSignalThreadGroup(tg, &arch.SignalInfo{Signo: signo}); err != nil {
lastErr = err
}
}
@@ -1055,33 +1211,57 @@ func (l *Loader) signalAllProcesses(cid string, signo int32) error {
return l.k.SendContainerSignal(cid, &arch.SignalInfo{Signo: signo})
}
-// threadGroupFromID same as threadGroupFromIDLocked except that it acquires
-// mutex before calling it.
-func (l *Loader) threadGroupFromID(key execID) (*kernel.ThreadGroup, *host.TTYFileOperations, error) {
+// threadGroupFromID is similar to tryThreadGroupFromIDLocked except that it
+// acquires mutex before calling it and fails in case container hasn't started
+// yet.
+func (l *Loader) threadGroupFromID(key execID) (*kernel.ThreadGroup, error) {
l.mu.Lock()
defer l.mu.Unlock()
- tg, tty, ok, err := l.threadGroupFromIDLocked(key)
+ tg, err := l.tryThreadGroupFromIDLocked(key)
if err != nil {
- return nil, nil, err
+ return nil, err
}
- if !ok {
- return nil, nil, fmt.Errorf("container %q not started", key.cid)
+ if tg == nil {
+ return nil, fmt.Errorf("container %q not started", key.cid)
}
- return tg, tty, nil
+ return tg, nil
}
-// threadGroupFromIDLocked returns the thread group and TTY for the given
-// execution ID. TTY may be nil if the process is not attached to a terminal.
-// Also returns a boolean indicating whether the container has already started.
-// Returns error if execution ID is invalid or if the container cannot be
-// found (maybe it has been deleted). Caller must hold 'mu'.
-func (l *Loader) threadGroupFromIDLocked(key execID) (*kernel.ThreadGroup, *host.TTYFileOperations, bool, error) {
+// tryThreadGroupFromIDLocked returns the thread group for the given execution
+// ID. It may return nil in case the container has not started yet. Returns
+// error if execution ID is invalid or if the container cannot be found (maybe
+// it has been deleted). Caller must hold 'mu'.
+func (l *Loader) tryThreadGroupFromIDLocked(key execID) (*kernel.ThreadGroup, error) {
ep := l.processes[key]
if ep == nil {
- return nil, nil, false, fmt.Errorf("container %q not found", key.cid)
+ return nil, fmt.Errorf("container %q not found", key.cid)
}
- if ep.tg == nil {
- return nil, nil, false, nil
+ return ep.tg, nil
+}
+
+// ttyFromIDLocked returns the TTY files for the given execution ID. It may
+// return nil in case the container has not started yet. Returns error if
+// execution ID is invalid or if the container cannot be found (maybe it has
+// been deleted). Caller must hold 'mu'.
+func (l *Loader) ttyFromIDLocked(key execID) (*host.TTYFileOperations, *hostvfs2.TTYFileDescription, error) {
+ ep := l.processes[key]
+ if ep == nil {
+ return nil, nil, fmt.Errorf("container %q not found", key.cid)
+ }
+ return ep.tty, ep.ttyVFS2, nil
+}
+
+func createFDTable(ctx context.Context, console bool, stdioFDs []int) (*kernel.FDTable, *host.TTYFileOperations, *hostvfs2.TTYFileDescription, error) {
+ if len(stdioFDs) != 3 {
+ return nil, nil, nil, fmt.Errorf("stdioFDs should contain exactly 3 FDs (stdin, stdout, and stderr), but %d FDs received", len(stdioFDs))
+ }
+
+ k := kernel.KernelFromContext(ctx)
+ fdTable := k.NewFDTable()
+ ttyFile, ttyFileVFS2, err := fdimport.Import(ctx, fdTable, console, stdioFDs)
+ if err != nil {
+ fdTable.DecRef()
+ return nil, nil, nil, err
}
- return ep.tg, ep.tty, true, nil
+ return fdTable, ttyFile, ttyFileVFS2, nil
}