diff options
Diffstat (limited to 'runsc')
-rw-r--r-- | runsc/boot/controller.go | 72 | ||||
-rw-r--r-- | runsc/boot/fs.go | 18 | ||||
-rw-r--r-- | runsc/boot/loader.go | 87 | ||||
-rw-r--r-- | runsc/boot/network.go | 33 | ||||
-rw-r--r-- | runsc/boot/vfs.go | 70 | ||||
-rw-r--r-- | runsc/cgroup/cgroup.go | 32 | ||||
-rw-r--r-- | runsc/cgroup/cgroup_test.go | 5 | ||||
-rw-r--r-- | runsc/cmd/do.go | 2 | ||||
-rw-r--r-- | runsc/cmd/exec.go | 2 | ||||
-rw-r--r-- | runsc/console/console.go | 4 | ||||
-rw-r--r-- | runsc/container/BUILD | 5 | ||||
-rw-r--r-- | runsc/container/console_test.go | 205 | ||||
-rw-r--r-- | runsc/container/container.go | 37 | ||||
-rw-r--r-- | runsc/container/multi_container_test.go | 53 | ||||
-rw-r--r-- | runsc/sandbox/network.go | 25 | ||||
-rw-r--r-- | runsc/sandbox/sandbox.go | 30 |
16 files changed, 493 insertions, 187 deletions
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index fdf13c8e1..865126ac5 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -211,10 +211,31 @@ func (cm *containerManager) Processes(cid *string, out *[]*control.Process) erro return control.Processes(cm.l.k, *cid, out) } +// CreateArgs contains arguments to the Create method. +type CreateArgs struct { + // CID is the ID of the container to start. + CID string + + // FilePayload may contain a TTY file for the terminal, if enabled. + urpc.FilePayload +} + // Create creates a container within a sandbox. -func (cm *containerManager) Create(cid *string, _ *struct{}) error { - log.Debugf("containerManager.Create, cid: %s", *cid) - return cm.l.createContainer(*cid) +func (cm *containerManager) Create(args *CreateArgs, _ *struct{}) error { + log.Debugf("containerManager.Create: %s", args.CID) + + if len(args.Files) > 1 { + return fmt.Errorf("start arguments must have at most 1 files for TTY") + } + var tty *fd.FD + if len(args.Files) == 1 { + var err error + tty, err = fd.NewFromFile(args.Files[0]) + if err != nil { + return fmt.Errorf("error dup'ing TTY file: %w", err) + } + } + return cm.l.createContainer(args.CID, tty) } // StartArgs contains arguments to the Start method. @@ -229,9 +250,8 @@ type StartArgs struct { CID string // FilePayload contains, in order: - // * stdin, stdout, and stderr. - // * the file descriptor over which the sandbox will - // request files from its root filesystem. + // * stdin, stdout, and stderr (optional: if terminal is disabled). + // * file descriptors to connect to gofer to serve the root filesystem. urpc.FilePayload } @@ -251,23 +271,45 @@ func (cm *containerManager) Start(args *StartArgs, _ *struct{}) error { if args.CID == "" { return errors.New("start argument missing container ID") } - if len(args.FilePayload.Files) < 4 { - return fmt.Errorf("start arguments must contain stdin, stderr, and stdout followed by at least one file for the container root gofer") + if len(args.Files) < 1 { + return fmt.Errorf("start arguments must contain at least one file for the container root gofer") } // All validation passed, logs the spec for debugging. specutils.LogSpec(args.Spec) - fds, err := fd.NewFromFiles(args.FilePayload.Files) + goferFiles := args.Files + var stdios []*fd.FD + if !args.Spec.Process.Terminal { + // When not using a terminal, stdios come as the first 3 files in the + // payload. + if l := len(args.Files); l < 4 { + return fmt.Errorf("start arguments (len: %d) must contain stdios and files for the container root gofer", l) + } + var err error + stdios, err = fd.NewFromFiles(goferFiles[:3]) + if err != nil { + return fmt.Errorf("error dup'ing stdio files: %w", err) + } + goferFiles = goferFiles[3:] + } + defer func() { + for _, fd := range stdios { + _ = fd.Close() + } + }() + + goferFDs, err := fd.NewFromFiles(goferFiles) if err != nil { - return err + return fmt.Errorf("error dup'ing gofer files: %w", err) } defer func() { - for _, fd := range fds { + for _, fd := range goferFDs { _ = fd.Close() } }() - if err := cm.l.startContainer(args.Spec, args.Conf, args.CID, fds); err != nil { + + if err := cm.l.startContainer(args.Spec, args.Conf, args.CID, stdios, goferFDs); err != nil { log.Debugf("containerManager.Start failed, cid: %s, args: %+v, err: %v", args.CID, args, err) return err } @@ -330,18 +372,18 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error { log.Debugf("containerManager.Restore") var specFile, deviceFile *os.File - switch numFiles := len(o.FilePayload.Files); numFiles { + switch numFiles := len(o.Files); numFiles { case 2: // The device file is donated to the platform. // Can't take ownership away from os.File. dup them to get a new FD. - fd, err := syscall.Dup(int(o.FilePayload.Files[1].Fd())) + fd, err := syscall.Dup(int(o.Files[1].Fd())) if err != nil { return fmt.Errorf("failed to dup file: %v", err) } deviceFile = os.NewFile(uintptr(fd), "platform device") fallthrough case 1: - specFile = o.FilePayload.Files[0] + specFile = o.Files[0] case 0: return fmt.Errorf("at least one file must be passed to Restore") default: diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index 6b6ae98d7..2b0d2cd51 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -22,15 +22,6 @@ import ( "strings" "syscall" - // Include filesystem types that OCI spec might mount. - _ "gvisor.dev/gvisor/pkg/sentry/fs/dev" - _ "gvisor.dev/gvisor/pkg/sentry/fs/host" - _ "gvisor.dev/gvisor/pkg/sentry/fs/proc" - _ "gvisor.dev/gvisor/pkg/sentry/fs/sys" - _ "gvisor.dev/gvisor/pkg/sentry/fs/tmpfs" - _ "gvisor.dev/gvisor/pkg/sentry/fs/tty" - "gvisor.dev/gvisor/pkg/sentry/vfs" - specs "github.com/opencontainers/runtime-spec/specs-go" "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" @@ -48,9 +39,18 @@ import ( tmpfsvfs2 "gvisor.dev/gvisor/pkg/sentry/fsimpl/tmpfs" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/runsc/config" "gvisor.dev/gvisor/runsc/specutils" + + // Include filesystem types that OCI spec might mount. + _ "gvisor.dev/gvisor/pkg/sentry/fs/dev" + _ "gvisor.dev/gvisor/pkg/sentry/fs/host" + _ "gvisor.dev/gvisor/pkg/sentry/fs/proc" + _ "gvisor.dev/gvisor/pkg/sentry/fs/sys" + _ "gvisor.dev/gvisor/pkg/sentry/fs/tmpfs" + _ "gvisor.dev/gvisor/pkg/sentry/fs/tty" ) const ( diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index ebdd518d0..3df013d34 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -157,6 +157,11 @@ type execProcess struct { // pidnsPath is the pid namespace path in spec pidnsPath string + + // hostTTY is present when creating a sub-container with terminal enabled. + // TTY file is passed during container create and must be saved until + // container start. + hostTTY *fd.FD } func init() { @@ -588,7 +593,9 @@ func (l *Loader) run() error { // Create the root container init task. It will begin running // when the kernel is started. - if _, err := l.createContainerProcess(true, l.sandboxID, &l.root, ep); err != nil { + var err error + _, ep.tty, ep.ttyVFS2, err = l.createContainerProcess(true, l.sandboxID, &l.root) + if err != nil { return err } @@ -627,7 +634,7 @@ func (l *Loader) run() error { } // createContainer creates a new container inside the sandbox. -func (l *Loader) createContainer(cid string) error { +func (l *Loader) createContainer(cid string, tty *fd.FD) error { l.mu.Lock() defer l.mu.Unlock() @@ -635,14 +642,14 @@ func (l *Loader) createContainer(cid string) error { if _, ok := l.processes[eid]; ok { return fmt.Errorf("container %q already exists", cid) } - l.processes[eid] = &execProcess{} + l.processes[eid] = &execProcess{hostTTY: tty} return nil } // startContainer starts a child container. It returns the thread group ID of // the newly created process. Used FDs are either closed or released. It's safe // for the caller to close any remaining files upon return. -func (l *Loader) startContainer(spec *specs.Spec, conf *config.Config, cid string, files []*fd.FD) error { +func (l *Loader) startContainer(spec *specs.Spec, conf *config.Config, cid string, stdioFDs, goferFDs []*fd.FD) error { // Create capabilities. caps, err := specutils.Capabilities(conf.EnableRaw, spec.Process.Capabilities) if err != nil { @@ -695,36 +702,41 @@ func (l *Loader) startContainer(spec *specs.Spec, conf *config.Config, cid strin info := &containerInfo{ conf: conf, spec: spec, - stdioFDs: files[:3], - goferFDs: files[3:], + goferFDs: goferFDs, } info.procArgs, err = createProcessArgs(cid, spec, creds, l.k, pidns) if err != nil { return fmt.Errorf("creating new process: %v", err) } - tg, err := l.createContainerProcess(false, cid, info, ep) + + // Use stdios or TTY depending on the spec configuration. + if spec.Process.Terminal { + if len(stdioFDs) > 0 { + return fmt.Errorf("using TTY, stdios not expected: %v", stdioFDs) + } + if ep.hostTTY == nil { + return fmt.Errorf("terminal enabled but no TTY provided. Did you set --console-socket on create?") + } + info.stdioFDs = []*fd.FD{ep.hostTTY, ep.hostTTY, ep.hostTTY} + ep.hostTTY = nil + } else { + info.stdioFDs = stdioFDs + } + + ep.tg, ep.tty, ep.ttyVFS2, err = l.createContainerProcess(false, cid, info) if err != nil { return err } - - // Success! - l.k.StartProcess(tg) - ep.tg = tg + l.k.StartProcess(ep.tg) return nil } -func (l *Loader) createContainerProcess(root bool, cid string, info *containerInfo, ep *execProcess) (*kernel.ThreadGroup, error) { - console := false - if root { - // Only root container supports terminal for now. - console = info.spec.Process.Terminal - } - +func (l *Loader) createContainerProcess(root bool, cid string, info *containerInfo) (*kernel.ThreadGroup, *host.TTYFileOperations, *hostvfs2.TTYFileDescription, error) { // Create the FD map, which will set stdin, stdout, and stderr. ctx := info.procArgs.NewContext(l.k) - fdTable, ttyFile, ttyFileVFS2, err := createFDTable(ctx, console, info.stdioFDs) + fdTable, ttyFile, ttyFileVFS2, err := createFDTable(ctx, info.spec.Process.Terminal, info.stdioFDs) if err != nil { - return nil, fmt.Errorf("importing fds: %v", err) + return nil, nil, nil, fmt.Errorf("importing fds: %v", err) } // CreateProcess takes a reference on fdTable if successful. We won't need // ours either way. @@ -736,11 +748,11 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn mntr := newContainerMounter(info.spec, info.goferFDs, l.k, l.mountHints) if root { if err := mntr.processHints(info.conf, info.procArgs.Credentials); err != nil { - return nil, err + return nil, nil, nil, err } } if err := setupContainerFS(ctx, info.conf, mntr, &info.procArgs); err != nil { - return nil, err + return nil, nil, nil, err } // Add the HOME environment variable if it is not already set. @@ -754,29 +766,25 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn info.procArgs.Credentials.RealKUID, info.procArgs.Envv) } if err != nil { - return nil, err + return nil, nil, nil, err } info.procArgs.Envv = envv // Create and start the new process. tg, _, err := l.k.CreateProcess(info.procArgs) if err != nil { - return nil, fmt.Errorf("creating process: %v", err) + return nil, nil, nil, fmt.Errorf("creating process: %v", err) } // CreateProcess takes a reference on FDTable if successful. info.procArgs.FDTable.DecRef(ctx) // Set the foreground process group on the TTY to the global init process // group, since that is what we are about to start running. - if root { - switch { - case ttyFileVFS2 != nil: - ep.ttyVFS2 = ttyFileVFS2 - ttyFileVFS2.InitForegroundProcessGroup(tg.ProcessGroup()) - case ttyFile != nil: - ep.tty = ttyFile - ttyFile.InitForegroundProcessGroup(tg.ProcessGroup()) - } + switch { + case ttyFileVFS2 != nil: + ttyFileVFS2.InitForegroundProcessGroup(tg.ProcessGroup()) + case ttyFile != nil: + ttyFile.InitForegroundProcessGroup(tg.ProcessGroup()) } // Install seccomp filters with the new task if there are any. @@ -784,7 +792,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn if info.spec.Linux != nil && info.spec.Linux.Seccomp != nil { program, err := seccomp.BuildProgram(info.spec.Linux.Seccomp) if err != nil { - return nil, fmt.Errorf("building seccomp program: %v", err) + return nil, nil, nil, fmt.Errorf("building seccomp program: %v", err) } if log.IsLogging(log.Debug) { @@ -795,7 +803,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn task := tg.Leader() // NOTE: It seems Flags are ignored by runc so we ignore them too. if err := task.AppendSyscallFilter(program, true); err != nil { - return nil, fmt.Errorf("appending seccomp filters: %v", err) + return nil, nil, nil, fmt.Errorf("appending seccomp filters: %v", err) } } } else { @@ -804,7 +812,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn } } - return tg, nil + return tg, ttyFile, ttyFileVFS2, nil } // startGoferMonitor runs a goroutine to monitor gofer's health. It polls on @@ -1074,7 +1082,12 @@ func newRootNetworkNamespace(conf *config.Config, clock tcpip.Clock, uniqueID st func newEmptySandboxNetworkStack(clock tcpip.Clock, uniqueID stack.UniqueID) (inet.Stack, error) { netProtos := []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol, arp.NewProtocol} - transProtos := []stack.TransportProtocolFactory{tcp.NewProtocol, udp.NewProtocol, icmp.NewProtocol4} + transProtos := []stack.TransportProtocolFactory{ + tcp.NewProtocol, + udp.NewProtocol, + icmp.NewProtocol4, + icmp.NewProtocol6, + } s := netstack.Stack{stack.New(stack.Options{ NetworkProtocols: netProtos, TransportProtocols: transProtos, diff --git a/runsc/boot/network.go b/runsc/boot/network.go index f58b09942..3d3a813df 100644 --- a/runsc/boot/network.go +++ b/runsc/boot/network.go @@ -40,9 +40,9 @@ var ( // "::1/8" on "lo" interface. DefaultLoopbackLink = LoopbackLink{ Name: "lo", - Addresses: []net.IP{ - net.IP("\x7f\x00\x00\x01"), - net.IPv6loopback, + Addresses: []IPWithPrefix{ + {Address: net.IP("\x7f\x00\x00\x01"), PrefixLen: 8}, + {Address: net.IPv6loopback, PrefixLen: 128}, }, Routes: []Route{ { @@ -82,7 +82,7 @@ type DefaultRoute struct { type FDBasedLink struct { Name string MTU int - Addresses []net.IP + Addresses []IPWithPrefix Routes []Route GSOMaxSize uint32 SoftwareGSOEnabled bool @@ -99,7 +99,7 @@ type FDBasedLink struct { // LoopbackLink configures a loopback li nk. type LoopbackLink struct { Name string - Addresses []net.IP + Addresses []IPWithPrefix Routes []Route } @@ -117,6 +117,19 @@ type CreateLinksAndRoutesArgs struct { Defaultv6Gateway DefaultRoute } +// IPWithPrefix is an address with its subnet prefix length. +type IPWithPrefix struct { + // Address is a network address. + Address net.IP + + // PrefixLen is the subnet prefix length. + PrefixLen int +} + +func (ip IPWithPrefix) String() string { + return fmt.Sprintf("%s/%d", ip.Address, ip.PrefixLen) +} + // Empty returns true if route hasn't been set. func (r *Route) Empty() bool { return r.Destination.IP == nil && r.Destination.Mask == nil && r.Gateway == nil @@ -264,15 +277,19 @@ func (n *Network) CreateLinksAndRoutes(args *CreateLinksAndRoutesArgs, _ *struct // createNICWithAddrs creates a NIC in the network stack and adds the given // addresses. -func (n *Network) createNICWithAddrs(id tcpip.NICID, name string, ep stack.LinkEndpoint, addrs []net.IP) error { +func (n *Network) createNICWithAddrs(id tcpip.NICID, name string, ep stack.LinkEndpoint, addrs []IPWithPrefix) error { opts := stack.NICOptions{Name: name} if err := n.Stack.CreateNICWithOptions(id, sniffer.New(ep), opts); err != nil { return fmt.Errorf("CreateNICWithOptions(%d, _, %+v) failed: %v", id, opts, err) } for _, addr := range addrs { - proto, tcpipAddr := ipToAddressAndProto(addr) - if err := n.Stack.AddAddress(id, proto, tcpipAddr); err != nil { + proto, tcpipAddr := ipToAddressAndProto(addr.Address) + ap := tcpip.AddressWithPrefix{ + Address: tcpipAddr, + PrefixLen: addr.PrefixLen, + } + if err := n.Stack.AddAddressWithPrefix(id, proto, ap); err != nil { return fmt.Errorf("AddAddress(%v, %v, %v) failed: %v", id, proto, tcpipAddr, err) } } diff --git a/runsc/boot/vfs.go b/runsc/boot/vfs.go index b157387ef..3fd28e516 100644 --- a/runsc/boot/vfs.go +++ b/runsc/boot/vfs.go @@ -250,36 +250,76 @@ func (c *containerMounter) configureOverlay(ctx context.Context, creds *auth.Cre overlayOpts := *lowerOpts overlayOpts.GetFilesystemOptions = vfs.GetFilesystemOptions{} - // Next mount upper and lower. Upper is a tmpfs mount to keep all - // modifications inside the sandbox. - upper, err := c.k.VFS().MountDisconnected(ctx, creds, "" /* source */, tmpfs.Name, &upperOpts) - if err != nil { - return nil, nil, fmt.Errorf("failed to create upper layer for overlay, opts: %+v: %v", upperOpts, err) - } - cu := cleanup.Make(func() { upper.DecRef(ctx) }) - defer cu.Clean() - // All writes go to the upper layer, be paranoid and make lower readonly. lowerOpts.ReadOnly = true lower, err := c.k.VFS().MountDisconnected(ctx, creds, "" /* source */, lowerFSName, lowerOpts) if err != nil { return nil, nil, err } - cu.Add(func() { lower.DecRef(ctx) }) + cu := cleanup.Make(func() { lower.DecRef(ctx) }) + defer cu.Clean() - // Propagate the lower layer's root's owner, group, and mode to the upper - // layer's root for consistency with VFS1. - upperRootVD := vfs.MakeVirtualDentry(upper, upper.Root()) + // Determine the lower layer's root's type. lowerRootVD := vfs.MakeVirtualDentry(lower, lower.Root()) stat, err := c.k.VFS().StatAt(ctx, creds, &vfs.PathOperation{ Root: lowerRootVD, Start: lowerRootVD, }, &vfs.StatOptions{ - Mask: linux.STATX_UID | linux.STATX_GID | linux.STATX_MODE, + Mask: linux.STATX_UID | linux.STATX_GID | linux.STATX_MODE | linux.STATX_TYPE, }) if err != nil { - return nil, nil, err + return nil, nil, fmt.Errorf("failed to stat lower layer's root: %v", err) + } + if stat.Mask&linux.STATX_TYPE == 0 { + return nil, nil, fmt.Errorf("failed to get file type of lower layer's root") + } + rootType := stat.Mode & linux.S_IFMT + if rootType != linux.S_IFDIR && rootType != linux.S_IFREG { + return nil, nil, fmt.Errorf("lower layer's root has unsupported file type %v", rootType) + } + + // Upper is a tmpfs mount to keep all modifications inside the sandbox. + upperOpts.GetFilesystemOptions.InternalData = tmpfs.FilesystemOpts{ + RootFileType: uint16(rootType), + } + upper, err := c.k.VFS().MountDisconnected(ctx, creds, "" /* source */, tmpfs.Name, &upperOpts) + if err != nil { + return nil, nil, fmt.Errorf("failed to create upper layer for overlay, opts: %+v: %v", upperOpts, err) + } + cu.Add(func() { upper.DecRef(ctx) }) + + // If the overlay mount consists of a regular file, copy up its contents + // from the lower layer, since in the overlay the otherwise-empty upper + // layer file will take precedence. + upperRootVD := vfs.MakeVirtualDentry(upper, upper.Root()) + if rootType == linux.S_IFREG { + lowerFD, err := c.k.VFS().OpenAt(ctx, creds, &vfs.PathOperation{ + Root: lowerRootVD, + Start: lowerRootVD, + }, &vfs.OpenOptions{ + Flags: linux.O_RDONLY, + }) + if err != nil { + return nil, nil, fmt.Errorf("failed to open lower layer root for copying: %v", err) + } + defer lowerFD.DecRef(ctx) + upperFD, err := c.k.VFS().OpenAt(ctx, creds, &vfs.PathOperation{ + Root: upperRootVD, + Start: upperRootVD, + }, &vfs.OpenOptions{ + Flags: linux.O_WRONLY, + }) + if err != nil { + return nil, nil, fmt.Errorf("failed to open upper layer root for copying: %v", err) + } + defer upperFD.DecRef(ctx) + if _, err := vfs.CopyRegularFileData(ctx, upperFD, lowerFD); err != nil { + return nil, nil, fmt.Errorf("failed to copy up overlay file: %v", err) + } } + + // Propagate the lower layer's root's owner, group, and mode to the upper + // layer's root for consistency with VFS1. err = c.k.VFS().SetStatAt(ctx, creds, &vfs.PathOperation{ Root: upperRootVD, Start: upperRootVD, diff --git a/runsc/cgroup/cgroup.go b/runsc/cgroup/cgroup.go index 5bd0afc52..e5294de55 100644 --- a/runsc/cgroup/cgroup.go +++ b/runsc/cgroup/cgroup.go @@ -234,7 +234,7 @@ func loadPathsHelper(cgroup io.Reader) (map[string]string, error) { type Cgroup struct { Name string `json:"name"` Parents map[string]string `json:"parents"` - Own bool `json:"own"` + Own map[string]bool `json:"own"` } // New creates a new Cgroup instance if the spec includes a cgroup path. @@ -251,9 +251,11 @@ func New(spec *specs.Spec) (*Cgroup, error) { return nil, fmt.Errorf("finding current cgroups: %w", err) } } + own := make(map[string]bool) return &Cgroup{ Name: spec.Linux.CgroupsPath, Parents: parents, + Own: own, }, nil } @@ -261,18 +263,8 @@ func New(spec *specs.Spec) (*Cgroup, error) { // already exists, it means that the caller has already provided a // pre-configured cgroups, and 'res' is ignored. func (c *Cgroup) Install(res *specs.LinuxResources) error { - if _, err := os.Stat(c.makePath("memory")); err == nil { - // If cgroup has already been created; it has been setup by caller. Don't - // make any changes to configuration, just join when sandbox/gofer starts. - log.Debugf("Using pre-created cgroup %q", c.Name) - return nil - } - log.Debugf("Creating cgroup %q", c.Name) - // Mark that cgroup resources are owned by me. - c.Own = true - // The Cleanup object cleans up partially created cgroups when an error occurs. // Errors occuring during cleanup itself are ignored. clean := cleanup.Make(func() { _ = c.Uninstall() }) @@ -280,6 +272,16 @@ func (c *Cgroup) Install(res *specs.LinuxResources) error { for key, cfg := range controllers { path := c.makePath(key) + if _, err := os.Stat(path); err == nil { + // If cgroup has already been created; it has been setup by caller. Don't + // make any changes to configuration, just join when sandbox/gofer starts. + log.Debugf("Using pre-created cgroup %q", path) + continue + } + + // Mark that cgroup resources are owned by me. + c.Own[key] = true + if err := os.MkdirAll(path, 0755); err != nil { if cfg.optional && errors.Is(err, syscall.EROFS) { log.Infof("Skipping cgroup %q", key) @@ -298,12 +300,12 @@ func (c *Cgroup) Install(res *specs.LinuxResources) error { // Uninstall removes the settings done in Install(). If cgroup path already // existed when Install() was called, Uninstall is a noop. func (c *Cgroup) Uninstall() error { - if !c.Own { - // cgroup is managed by caller, don't touch it. - return nil - } log.Debugf("Deleting cgroup %q", c.Name) for key := range controllers { + if !c.Own[key] { + // cgroup is managed by caller, don't touch it. + continue + } path := c.makePath(key) log.Debugf("Removing cgroup controller for key=%q path=%q", key, path) diff --git a/runsc/cgroup/cgroup_test.go b/runsc/cgroup/cgroup_test.go index 9794517a7..931144cf9 100644 --- a/runsc/cgroup/cgroup_test.go +++ b/runsc/cgroup/cgroup_test.go @@ -29,7 +29,10 @@ func TestUninstallEnoent(t *testing.T) { c := Cgroup{ // set a non-existent name Name: "runsc-test-uninstall-656e6f656e740a", - Own: true, + } + c.Own = make(map[string]bool) + for key := range controllers { + c.Own[key] = true } if err := c.Uninstall(); err != nil { t.Errorf("Uninstall() failed: %v", err) diff --git a/runsc/cmd/do.go b/runsc/cmd/do.go index 640de4c47..8a8d9f752 100644 --- a/runsc/cmd/do.go +++ b/runsc/cmd/do.go @@ -81,7 +81,7 @@ func (c *Do) SetFlags(f *flag.FlagSet) { // Execute implements subcommands.Command.Execute. func (c *Do) Execute(_ context.Context, f *flag.FlagSet, args ...interface{}) subcommands.ExitStatus { if len(f.Args()) == 0 { - c.Usage() + f.Usage() return subcommands.ExitUsageError } diff --git a/runsc/cmd/exec.go b/runsc/cmd/exec.go index 86c02a22a..eafd6285c 100644 --- a/runsc/cmd/exec.go +++ b/runsc/cmd/exec.go @@ -150,7 +150,7 @@ func (ex *Exec) Execute(_ context.Context, f *flag.FlagSet, args ...interface{}) } func (ex *Exec) exec(c *container.Container, e *control.ExecArgs, waitStatus *syscall.WaitStatus) subcommands.ExitStatus { - // Start the new process and get it pid. + // Start the new process and get its pid. pid, err := c.Execute(e) if err != nil { return Errorf("executing processes for container: %v", err) diff --git a/runsc/console/console.go b/runsc/console/console.go index dbb88e117..b36028792 100644 --- a/runsc/console/console.go +++ b/runsc/console/console.go @@ -24,8 +24,8 @@ import ( "golang.org/x/sys/unix" ) -// NewWithSocket creates pty master/replica pair, sends the master FD over the given -// socket, and returns the replica. +// NewWithSocket creates pty master/replica pair, sends the master FD over the +// given socket, and returns the replica. func NewWithSocket(socketPath string) (*os.File, error) { // Create a new pty master and replica. ptyMaster, ptyReplica, err := pty.Open() diff --git a/runsc/container/BUILD b/runsc/container/BUILD index c33755482..8793c8916 100644 --- a/runsc/container/BUILD +++ b/runsc/container/BUILD @@ -1,4 +1,4 @@ -load("//tools:defs.bzl", "go_library", "go_test") +load("//tools:defs.bzl", "go_library", "go_test", "more_shards") package(licenses = ["notice"]) @@ -24,6 +24,7 @@ go_library( "//runsc/boot", "//runsc/cgroup", "//runsc/config", + "//runsc/console", "//runsc/sandbox", "//runsc/specutils", "@com_github_cenkalti_backoff//:go_default_library", @@ -48,7 +49,7 @@ go_test( "//test/cmd/test_app", ], library = ":container", - shard_count = 10, + shard_count = more_shards, tags = [ "requires-kvm", ], diff --git a/runsc/container/console_test.go b/runsc/container/console_test.go index 4228399b8..1b0fdebd6 100644 --- a/runsc/container/console_test.go +++ b/runsc/container/console_test.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "io" + "math/rand" "os" "path/filepath" "syscall" @@ -27,7 +28,6 @@ import ( "github.com/kr/pty" "golang.org/x/sys/unix" "gvisor.dev/gvisor/pkg/sentry/control" - "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/test/testutil" "gvisor.dev/gvisor/pkg/unet" @@ -38,19 +38,22 @@ import ( // path is under 108 charactors (the unix socket path length limit), // relativizing the path if necessary. func socketPath(bundleDir string) (string, error) { - path := filepath.Join(bundleDir, "socket") + num := rand.Intn(10000) + path := filepath.Join(bundleDir, fmt.Sprintf("socket-%4d", num)) + const maxPathLen = 108 + if len(path) <= maxPathLen { + return path, nil + } + + // Path is too large, try to make it smaller. cwd, err := os.Getwd() if err != nil { return "", fmt.Errorf("error getting cwd: %v", err) } - relPath, err := filepath.Rel(cwd, path) + path, err = filepath.Rel(cwd, path) if err != nil { return "", fmt.Errorf("error getting relative path for %q from cwd %q: %v", path, cwd, err) } - if len(path) > len(relPath) { - path = relPath - } - const maxPathLen = 108 if len(path) > maxPathLen { return "", fmt.Errorf("could not get socket path under length limit %d: %s", maxPathLen, path) } @@ -159,6 +162,82 @@ func TestConsoleSocket(t *testing.T) { } } +// Test that an pty FD is sent over the console socket if one is provided. +func TestMultiContainerConsoleSocket(t *testing.T) { + for name, conf := range configsWithVFS2(t, all...) { + t.Run(name, func(t *testing.T) { + rootDir, cleanup, err := testutil.SetupRootDir() + if err != nil { + t.Fatalf("error creating root dir: %v", err) + } + defer cleanup() + conf.RootDir = rootDir + + // Setup the containers. + sleep := []string{"sleep", "100"} + tru := []string{"true"} + testSpecs, ids := createSpecs(sleep, tru) + testSpecs[1].Process.Terminal = true + + bundleDir, cleanup, err := testutil.SetupBundleDir(testSpecs[0]) + if err != nil { + t.Fatalf("error setting up container: %v", err) + } + defer cleanup() + + args := Args{ + ID: ids[0], + Spec: testSpecs[0], + BundleDir: bundleDir, + } + rootCont, err := New(conf, args) + if err != nil { + t.Fatalf("error creating container: %v", err) + } + defer rootCont.Destroy() + if err := rootCont.Start(conf); err != nil { + t.Fatalf("error starting container: %v", err) + } + + bundleDir, cleanup, err = testutil.SetupBundleDir(testSpecs[0]) + if err != nil { + t.Fatalf("error setting up container: %v", err) + } + defer cleanup() + + sock, err := socketPath(bundleDir) + if err != nil { + t.Fatalf("error getting socket path: %v", err) + } + srv, cleanup := createConsoleSocket(t, sock) + defer cleanup() + + // Create the container and pass the socket name. + args = Args{ + ID: ids[1], + Spec: testSpecs[1], + BundleDir: bundleDir, + ConsoleSocket: sock, + } + cont, err := New(conf, args) + 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) + } + + // 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") @@ -221,9 +300,9 @@ func TestJobControlSignalExec(t *testing.T) { // Make sure all the processes are running. expectedPL := []*control.Process{ // Root container process. - {PID: 1, Cmd: "sleep", Threads: []kernel.ThreadID{1}}, + newProcessBuilder().Cmd("sleep").Process(), // Bash from exec process. - {PID: 2, Cmd: "bash", Threads: []kernel.ThreadID{2}}, + newProcessBuilder().PID(2).Cmd("bash").Process(), } if err := waitForProcessList(c, expectedPL); err != nil { t.Error(err) @@ -233,7 +312,7 @@ func TestJobControlSignalExec(t *testing.T) { 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", Threads: []kernel.ThreadID{3}}) + expectedPL = append(expectedPL, newProcessBuilder().PID(3).PPID(2).Cmd("sleep").Process()) if err := waitForProcessList(c, expectedPL); err != nil { t.Error(err) } @@ -254,7 +333,7 @@ func TestJobControlSignalExec(t *testing.T) { // 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 { + if err := testutil.WaitUntilRead(ptyMaster, "Terminated", 5*time.Second); err != nil { t.Fatalf("bash did not take over pty: %v", err) } @@ -359,7 +438,7 @@ func TestJobControlSignalRootContainer(t *testing.T) { // Wait for bash to start. expectedPL := []*control.Process{ - {PID: 1, Cmd: "bash", Threads: []kernel.ThreadID{1}}, + newProcessBuilder().PID(1).Cmd("bash").Process(), } if err := waitForProcessList(c, expectedPL); err != nil { t.Fatalf("error waiting for processes: %v", err) @@ -369,7 +448,7 @@ func TestJobControlSignalRootContainer(t *testing.T) { ptyMaster.Write([]byte("sleep 100\n")) // Wait for sleep to start. - expectedPL = append(expectedPL, &control.Process{PID: 2, PPID: 1, Cmd: "sleep", Threads: []kernel.ThreadID{2}}) + expectedPL = append(expectedPL, newProcessBuilder().PID(2).PPID(1).Cmd("sleep").Process()) if err := waitForProcessList(c, expectedPL); err != nil { t.Fatalf("error waiting for processes: %v", err) } @@ -393,7 +472,7 @@ func TestJobControlSignalRootContainer(t *testing.T) { // 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 { + if err := testutil.WaitUntilRead(ptyBuf, "Terminated", 5*time.Second); err != nil { t.Fatalf("bash did not take over pty: %v", err) } @@ -414,6 +493,104 @@ func TestJobControlSignalRootContainer(t *testing.T) { } } +// Test that terminal works with root and sub-containers. +func TestMultiContainerTerminal(t *testing.T) { + for name, conf := range configsWithVFS2(t, all...) { + t.Run(name, func(t *testing.T) { + rootDir, cleanup, err := testutil.SetupRootDir() + if err != nil { + t.Fatalf("error creating root dir: %v", err) + } + defer cleanup() + conf.RootDir = rootDir + + // Don't let bash execute from profile or rc files, otherwise our PID + // counts get messed up. + bash := []string{"/bin/bash", "--noprofile", "--norc"} + testSpecs, ids := createSpecs(bash, bash) + + type termContainer struct { + container *Container + master *os.File + } + var containers []termContainer + for i, spec := range testSpecs { + bundleDir, cleanup, err := testutil.SetupBundleDir(spec) + if err != nil { + t.Fatalf("error setting up container: %v", err) + } + defer cleanup() + + spec.Process.Terminal = true + sock, err := socketPath(bundleDir) + if err != nil { + t.Fatalf("error getting socket path: %v", err) + } + srv, cleanup := createConsoleSocket(t, sock) + defer cleanup() + + // Create the container and pass the socket name. + args := Args{ + ID: ids[i], + Spec: spec, + BundleDir: bundleDir, + ConsoleSocket: sock, + } + cont, err := New(conf, args) + 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) + } + + // Make sure we get a console PTY. + ptyMaster, err := receiveConsolePTY(srv) + if err != nil { + t.Fatalf("error receiving console FD: %v", err) + } + defer ptyMaster.Close() + + containers = append(containers, termContainer{ + container: cont, + master: ptyMaster, + }) + } + + for _, tc := range containers { + // 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 stderr 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(tc.master, ptyBuf) + go io.Copy(os.Stderr, tee) + + // Wait for bash to start. + expectedPL := []*control.Process{ + newProcessBuilder().Cmd("bash").Process(), + } + if err := waitForProcessList(tc.container, expectedPL); err != nil { + t.Fatalf("error waiting for processes: %v", err) + } + + // Execute echo command and check that it was executed correctly. Use + // a variable to ensure it's not matching against command echo. + tc.master.Write([]byte("echo foo-${PWD}-123\n")) + if err := testutil.WaitUntilRead(ptyBuf, "foo-/-123", 5*time.Second); err != nil { + t.Fatalf("echo didn't execute: %v", err) + } + } + }) + } +} + // blockingBuffer is a thread-safe buffer that blocks when reading if the // buffer is empty. It implements io.ReadWriter. type blockingBuffer struct { diff --git a/runsc/container/container.go b/runsc/container/container.go index 4aa139c88..418a27beb 100644 --- a/runsc/container/container.go +++ b/runsc/container/container.go @@ -38,6 +38,7 @@ import ( "gvisor.dev/gvisor/runsc/boot" "gvisor.dev/gvisor/runsc/cgroup" "gvisor.dev/gvisor/runsc/config" + "gvisor.dev/gvisor/runsc/console" "gvisor.dev/gvisor/runsc/sandbox" "gvisor.dev/gvisor/runsc/specutils" ) @@ -79,6 +80,7 @@ func validateID(id string) error { // - It calls 'runsc delete'. runc implementation kills --all SIGKILL once // again just to be sure, waits, and then proceeds with remaining teardown. // +// Container is thread-unsafe. type Container struct { // ID is the container ID. ID string `json:"id"` @@ -397,7 +399,22 @@ func New(conf *config.Config, args Args) (*Container, error) { return nil, err } c.Sandbox = sb.Sandbox - if err := c.Sandbox.CreateContainer(c.ID); err != nil { + + // If the console control socket file is provided, then create a new + // pty master/slave pair and send the TTY to the sandbox process. + var tty *os.File + if c.ConsoleSocket != "" { + // Create a new TTY pair and send the master on the provided socket. + var err error + tty, err = console.NewWithSocket(c.ConsoleSocket) + if err != nil { + return nil, fmt.Errorf("setting up console with socket %q: %w", c.ConsoleSocket, err) + } + // tty file is transferred to the sandbox, then it can be closed here. + defer tty.Close() + } + + if err := c.Sandbox.CreateContainer(c.ID, tty); err != nil { return nil, err } } @@ -451,11 +468,16 @@ func (c *Container) Start(conf *config.Config) error { // the start (and all their children processes). if err := runInCgroup(c.Sandbox.Cgroup, func() error { // Create the gofer process. - ioFiles, mountsFile, err := c.createGoferProcess(c.Spec, conf, c.BundleDir, false) + goferFiles, mountsFile, err := c.createGoferProcess(c.Spec, conf, c.BundleDir, false) if err != nil { return err } - defer mountsFile.Close() + defer func() { + _ = mountsFile.Close() + for _, f := range goferFiles { + _ = f.Close() + } + }() cleanMounts, err := specutils.ReadMounts(mountsFile) if err != nil { @@ -463,7 +485,14 @@ func (c *Container) Start(conf *config.Config) error { } c.Spec.Mounts = cleanMounts - return c.Sandbox.StartContainer(c.Spec, conf, c.ID, ioFiles) + // Setup stdios if the container is not using terminal. Otherwise TTY was + // already setup in create. + var stdios []*os.File + if !c.Spec.Process.Terminal { + stdios = []*os.File{os.Stdin, os.Stdout, os.Stderr} + } + + return c.Sandbox.StartContainer(c.Spec, conf, c.ID, stdios, goferFiles) }); err != nil { return err } diff --git a/runsc/container/multi_container_test.go b/runsc/container/multi_container_test.go index cadc63bf3..45d4e6e6e 100644 --- a/runsc/container/multi_container_test.go +++ b/runsc/container/multi_container_test.go @@ -301,54 +301,21 @@ func TestMultiContainerWait(t *testing.T) { } defer cleanup() - // Check via ps that multiple processes are running. - expectedPL := []*control.Process{ - newProcessBuilder().PID(2).PPID(0).Cmd("sleep").Process(), - } - if err := waitForProcessList(containers[1], expectedPL); err != nil { - t.Errorf("failed to wait for sleep to start: %v", err) - } - - // Wait on the short lived container from multiple goroutines. - wg := sync.WaitGroup{} - for i := 0; i < 3; i++ { - wg.Add(1) - go func(c *Container) { - defer wg.Done() - if ws, err := c.Wait(); err != nil { - t.Errorf("failed to wait for process %s: %v", c.Spec.Process.Args, err) - } else if es := ws.ExitStatus(); es != 0 { - t.Errorf("process %s exited with non-zero status %d", c.Spec.Process.Args, es) - } - if _, err := c.Wait(); err != nil { - t.Errorf("wait for stopped container %s shouldn't fail: %v", c.Spec.Process.Args, err) - } - }(containers[1]) + // Check that we can wait for the sub-container. + c := containers[1] + if ws, err := c.Wait(); err != nil { + t.Errorf("failed to wait for process %s: %v", c.Spec.Process.Args, err) + } else if es := ws.ExitStatus(); es != 0 { + t.Errorf("process %s exited with non-zero status %d", c.Spec.Process.Args, es) } - - // Also wait via PID. - for i := 0; i < 3; i++ { - wg.Add(1) - go func(c *Container) { - defer wg.Done() - const pid = 2 - if ws, err := c.WaitPID(pid); err != nil { - t.Errorf("failed to wait for PID %d: %v", pid, err) - } else if es := ws.ExitStatus(); es != 0 { - t.Errorf("PID %d exited with non-zero status %d", pid, es) - } - if _, err := c.WaitPID(pid); err == nil { - t.Errorf("wait for stopped PID %d should fail", pid) - } - }(containers[1]) + if _, err := c.Wait(); err != nil { + t.Errorf("wait for stopped container %s shouldn't fail: %v", c.Spec.Process.Args, err) } - wg.Wait() - // After Wait returns, ensure that the root container is running and // the child has finished. - expectedPL = []*control.Process{ - newProcessBuilder().Cmd("sleep").Process(), + expectedPL := []*control.Process{ + newProcessBuilder().Cmd("sleep").PID(1).Process(), } 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) diff --git a/runsc/sandbox/network.go b/runsc/sandbox/network.go index 8f66dd1f8..d8112e7a2 100644 --- a/runsc/sandbox/network.go +++ b/runsc/sandbox/network.go @@ -127,7 +127,7 @@ func createInterfacesAndRoutesFromNS(conn *urpc.Client, nsPath string, hardwareG // Get all interfaces in the namespace. ifaces, err := net.Interfaces() if err != nil { - return fmt.Errorf("querying interfaces: %v", err) + return fmt.Errorf("querying interfaces: %w", err) } isRoot, err := isRootNS() @@ -148,14 +148,14 @@ func createInterfacesAndRoutesFromNS(conn *urpc.Client, nsPath string, hardwareG allAddrs, err := iface.Addrs() if err != nil { - return fmt.Errorf("fetching interface addresses for %q: %v", iface.Name, err) + return fmt.Errorf("fetching interface addresses for %q: %w", iface.Name, err) } // We build our own loopback device. if iface.Flags&net.FlagLoopback != 0 { link, err := loopbackLink(iface, allAddrs) if err != nil { - return fmt.Errorf("getting loopback link for iface %q: %v", iface.Name, err) + return fmt.Errorf("getting loopback link for iface %q: %w", iface.Name, err) } args.LoopbackLinks = append(args.LoopbackLinks, link) continue @@ -209,7 +209,7 @@ func createInterfacesAndRoutesFromNS(conn *urpc.Client, nsPath string, hardwareG // Get the link for the interface. ifaceLink, err := netlink.LinkByName(iface.Name) if err != nil { - return fmt.Errorf("getting link for interface %q: %v", iface.Name, err) + return fmt.Errorf("getting link for interface %q: %w", iface.Name, err) } link.LinkAddress = ifaceLink.Attrs().HardwareAddr @@ -219,7 +219,7 @@ func createInterfacesAndRoutesFromNS(conn *urpc.Client, nsPath string, hardwareG log.Debugf("Creating Channel %d", i) socketEntry, err := createSocket(iface, ifaceLink, hardwareGSO) if err != nil { - return fmt.Errorf("failed to createSocket for %s : %v", iface.Name, err) + return fmt.Errorf("failed to createSocket for %s : %w", iface.Name, err) } if i == 0 { link.GSOMaxSize = socketEntry.gsoMaxSize @@ -241,11 +241,12 @@ func createInterfacesAndRoutesFromNS(conn *urpc.Client, nsPath string, hardwareG // Collect the addresses for the interface, enable forwarding, // and remove them from the host. for _, addr := range ipAddrs { - link.Addresses = append(link.Addresses, addr.IP) + prefix, _ := addr.Mask.Size() + link.Addresses = append(link.Addresses, boot.IPWithPrefix{Address: addr.IP, PrefixLen: prefix}) // Steal IP address from NIC. if err := removeAddress(ifaceLink, addr.String()); err != nil { - return fmt.Errorf("removing address %v from device %q: %v", iface.Name, addr, err) + return fmt.Errorf("removing address %v from device %q: %w", addr, iface.Name, err) } } @@ -254,7 +255,7 @@ func createInterfacesAndRoutesFromNS(conn *urpc.Client, nsPath string, hardwareG log.Debugf("Setting up network, config: %+v", args) if err := conn.Call(boot.NetworkCreateLinksAndRoutes, &args, nil); err != nil { - return fmt.Errorf("creating links and routes: %v", err) + return fmt.Errorf("creating links and routes: %w", err) } return nil } @@ -339,9 +340,15 @@ func loopbackLink(iface net.Interface, addrs []net.Addr) (boot.LoopbackLink, err if !ok { return boot.LoopbackLink{}, fmt.Errorf("address is not IPNet: %+v", addr) } + + prefix, _ := ipNet.Mask.Size() + link.Addresses = append(link.Addresses, boot.IPWithPrefix{ + Address: ipNet.IP, + PrefixLen: prefix, + }) + dst := *ipNet dst.IP = dst.IP.Mask(dst.Mask) - link.Addresses = append(link.Addresses, ipNet.IP) link.Routes = append(link.Routes, boot.Route{ Destination: dst, }) diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 4a4110477..c84ebcd8a 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -173,7 +173,7 @@ func New(conf *config.Config, args *Args) (*Sandbox, error) { } // CreateContainer creates a non-root container inside the sandbox. -func (s *Sandbox) CreateContainer(cid string) error { +func (s *Sandbox) CreateContainer(cid string, tty *os.File) error { log.Debugf("Create non-root container %q in sandbox %q, PID: %d", cid, s.ID, s.Pid) sandboxConn, err := s.sandboxConnect() if err != nil { @@ -181,7 +181,16 @@ func (s *Sandbox) CreateContainer(cid string) error { } defer sandboxConn.Close() - if err := sandboxConn.Call(boot.ContainerCreate, &cid, nil); err != nil { + var files []*os.File + if tty != nil { + files = []*os.File{tty} + } + + args := boot.CreateArgs{ + CID: cid, + FilePayload: urpc.FilePayload{Files: files}, + } + if err := sandboxConn.Call(boot.ContainerCreate, &args, nil); err != nil { return fmt.Errorf("creating non-root container %q: %v", cid, err) } return nil @@ -211,11 +220,7 @@ func (s *Sandbox) StartRoot(spec *specs.Spec, conf *config.Config) error { } // StartContainer starts running a non-root container inside the sandbox. -func (s *Sandbox) StartContainer(spec *specs.Spec, conf *config.Config, cid string, goferFiles []*os.File) error { - for _, f := range goferFiles { - defer f.Close() - } - +func (s *Sandbox) StartContainer(spec *specs.Spec, conf *config.Config, cid string, stdios, goferFiles []*os.File) error { log.Debugf("Start non-root container %q in sandbox %q, PID: %d", cid, s.ID, s.Pid) sandboxConn, err := s.sandboxConnect() if err != nil { @@ -223,15 +228,18 @@ func (s *Sandbox) StartContainer(spec *specs.Spec, conf *config.Config, cid stri } defer sandboxConn.Close() - // The payload must container stdin/stdout/stderr followed by gofer - // files. - files := append([]*os.File{os.Stdin, os.Stdout, os.Stderr}, goferFiles...) + // The payload must contain stdin/stdout/stderr (which may be empty if using + // TTY) followed by gofer files. + payload := urpc.FilePayload{} + payload.Files = append(payload.Files, stdios...) + payload.Files = append(payload.Files, goferFiles...) + // Start running the container. args := boot.StartArgs{ Spec: spec, Conf: conf, CID: cid, - FilePayload: urpc.FilePayload{Files: files}, + FilePayload: payload, } if err := sandboxConn.Call(boot.ContainerStart, &args, nil); err != nil { return fmt.Errorf("starting non-root container %v: %v", spec.Process.Args, err) |