summaryrefslogtreecommitdiffhomepage
path: root/runsc/boot
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2019-06-04 06:19:01 +0000
committergVisor bot <gvisor-bot@google.com>2019-06-04 06:19:01 +0000
commit4f56f1bf2248bb17da8b269b4191218d85ce6587 (patch)
tree00e4c90f29024c89477bc7cee708f8020ff4a49a /runsc/boot
parentfcaa9421355cf770cf47c792388e25700b802e05 (diff)
parentf520d0d585e159da902b2880c5e115abeaacf9cb (diff)
Merge f520d0d5 (automated)
Diffstat (limited to 'runsc/boot')
-rw-r--r--runsc/boot/controller.go16
-rw-r--r--runsc/boot/fds.go3
-rw-r--r--runsc/boot/fs.go727
-rw-r--r--runsc/boot/loader.go73
4 files changed, 414 insertions, 405 deletions
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go
index 72ab9ef86..a277145b1 100644
--- a/runsc/boot/controller.go
+++ b/runsc/boot/controller.go
@@ -237,7 +237,7 @@ func (cm *containerManager) Start(args *StartArgs, _ *struct{}) error {
return fmt.Errorf("start arguments must contain stdin, stderr, and stdout followed by at least one file for the container root gofer")
}
- err := cm.l.startContainer(cm.l.k, args.Spec, args.Conf, args.CID, args.FilePayload.Files)
+ err := cm.l.startContainer(args.Spec, args.Conf, args.CID, args.FilePayload.Files)
if err != nil {
log.Debugf("containerManager.Start failed %q: %+v: %v", args.CID, args, err)
return err
@@ -340,8 +340,8 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error {
cm.l.k = k
// Set up the restore environment.
- fds := &fdDispenser{fds: cm.l.goferFDs}
- renv, err := createRestoreEnvironment(cm.l.spec, cm.l.conf, fds)
+ mntr := newContainerMounter(cm.l.spec, "", cm.l.goferFDs, cm.l.k)
+ renv, err := mntr.createRestoreEnvironment(cm.l.conf)
if err != nil {
return fmt.Errorf("creating RestoreEnvironment: %v", err)
}
@@ -369,11 +369,11 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error {
k.Timekeeper().SetClocks(time.NewCalibratedClocks())
// Since we have a new kernel we also must make a new watchdog.
- watchdog := watchdog.New(k, watchdog.DefaultTimeout, cm.l.conf.WatchdogAction)
+ dog := watchdog.New(k, watchdog.DefaultTimeout, cm.l.conf.WatchdogAction)
// Change the loader fields to reflect the changes made when restoring.
cm.l.k = k
- cm.l.watchdog = watchdog
+ cm.l.watchdog = dog
cm.l.rootProcArgs = kernel.CreateProcessArgs{}
cm.l.restore = true
@@ -420,16 +420,12 @@ type WaitPIDArgs struct {
// CID is the container ID.
CID string
-
- // ClearStatus determines whether the exit status of the process should
- // be cleared when WaitPID returns.
- ClearStatus bool
}
// WaitPID waits for the process with PID 'pid' in the sandbox.
func (cm *containerManager) WaitPID(args *WaitPIDArgs, waitStatus *uint32) error {
log.Debugf("containerManager.Wait")
- return cm.l.waitPID(kernel.ThreadID(args.PID), args.CID, args.ClearStatus, waitStatus)
+ return cm.l.waitPID(kernel.ThreadID(args.PID), args.CID, waitStatus)
}
// SignalDeliveryMode enumerates different signal delivery modes.
diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go
index 4e428b49c..0811e10f4 100644
--- a/runsc/boot/fds.go
+++ b/runsc/boot/fds.go
@@ -28,11 +28,12 @@ import (
// createFDMap creates an FD map that contains stdin, stdout, and stderr. If
// console is true, then ioctl calls will be passed through to the host FD.
// Upon success, createFDMap dups then closes stdioFDs.
-func createFDMap(ctx context.Context, k *kernel.Kernel, l *limits.LimitSet, console bool, stdioFDs []int) (*kernel.FDMap, error) {
+func createFDMap(ctx context.Context, l *limits.LimitSet, console bool, stdioFDs []int) (*kernel.FDMap, error) {
if len(stdioFDs) != 3 {
return nil, fmt.Errorf("stdioFDs should contain exactly 3 FDs (stdin, stdout, and stderr), but %d FDs received", len(stdioFDs))
}
+ k := kernel.KernelFromContext(ctx)
fdm := k.NewFDMap()
defer fdm.DecRef()
mounter := fs.FileOwnerFromContext(ctx)
diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go
index 4b1557b9a..939f2419c 100644
--- a/runsc/boot/fs.go
+++ b/runsc/boot/fs.go
@@ -29,9 +29,6 @@ import (
_ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/sys"
_ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/tmpfs"
_ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/tty"
- "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
- "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
- "gvisor.googlesource.com/gvisor/pkg/sentry/limits"
specs "github.com/opencontainers/runtime-spec/specs-go"
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
@@ -40,6 +37,8 @@ import (
"gvisor.googlesource.com/gvisor/pkg/sentry/fs"
"gvisor.googlesource.com/gvisor/pkg/sentry/fs/gofer"
"gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
"gvisor.googlesource.com/gvisor/pkg/syserror"
"gvisor.googlesource.com/gvisor/runsc/specutils"
)
@@ -65,67 +64,24 @@ const (
nonefs = "none"
)
-type fdDispenser struct {
- fds []int
-}
-
-func (f *fdDispenser) remove() int {
- if f.empty() {
- panic("fdDispenser out of fds")
- }
- rv := f.fds[0]
- f.fds = f.fds[1:]
- return rv
-}
-
-func (f *fdDispenser) empty() bool {
- return len(f.fds) == 0
-}
+func addOverlay(ctx context.Context, conf *Config, lower *fs.Inode, name string, lowerFlags fs.MountSourceFlags) (*fs.Inode, error) {
+ // Upper layer uses the same flags as lower, but it must be read-write.
+ upperFlags := lowerFlags
+ upperFlags.ReadOnly = false
-func adjustDirentCache(k *kernel.Kernel) error {
- var hl syscall.Rlimit
- if err := syscall.Getrlimit(syscall.RLIMIT_NOFILE, &hl); err != nil {
- return fmt.Errorf("getting RLIMIT_NOFILE: %v", err)
- }
- if int64(hl.Cur) != syscall.RLIM_INFINITY {
- newSize := hl.Cur / 2
- if newSize < gofer.DefaultDirentCacheSize {
- log.Infof("Setting gofer dirent cache size to %d", newSize)
- gofer.DefaultDirentCacheSize = newSize
- k.DirentCacheLimiter = fs.NewDirentCacheLimiter(newSize)
- }
+ tmpFS := mustFindFilesystem("tmpfs")
+ if !fs.IsDir(lower.StableAttr) {
+ // Create overlay on top of mount file, e.g. /etc/hostname.
+ msrc := fs.NewCachingMountSource(tmpFS, upperFlags)
+ return fs.NewOverlayRootFile(ctx, msrc, lower, upperFlags)
}
- return nil
-}
-
-// setupRootContainerFS creates a mount namespace containing the root filesystem
-// and all mounts. 'rootCtx' is used to walk directories to find mount points.
-// 'setMountNS' is called after namespace is created. It must set the mount NS
-// to 'rootCtx'.
-func setupRootContainerFS(userCtx context.Context, rootCtx context.Context, spec *specs.Spec, conf *Config, goferFDs []int, setMountNS func(*fs.MountNamespace)) error {
- mounts := compileMounts(spec)
-
- // Create a tmpfs mount where we create and mount a root filesystem for
- // each child container.
- mounts = append(mounts, specs.Mount{
- Type: tmpfs,
- Destination: ChildContainersDir,
- })
- fds := &fdDispenser{fds: goferFDs}
- rootInode, err := createRootMount(rootCtx, spec, conf, fds, mounts)
- if err != nil {
- return fmt.Errorf("creating root mount: %v", err)
- }
- mns, err := fs.NewMountNamespace(userCtx, rootInode)
+ // Create overlay on top of mount dir.
+ upper, err := tmpFS.Mount(ctx, name+"-upper", upperFlags, "", nil)
if err != nil {
- return fmt.Errorf("creating root mount namespace: %v", err)
+ return nil, fmt.Errorf("creating tmpfs overlay: %v", err)
}
- setMountNS(mns)
-
- root := mns.Root()
- defer root.DecRef()
- return mountSubmounts(rootCtx, conf, mns, root, mounts, fds)
+ return fs.NewOverlayRoot(ctx, upper, lower, upperFlags)
}
// compileMounts returns the supported mounts from the mount spec, adding any
@@ -184,186 +140,6 @@ func compileMounts(spec *specs.Spec) []specs.Mount {
return mounts
}
-// createRootMount creates the root filesystem.
-func createRootMount(ctx context.Context, spec *specs.Spec, conf *Config, fds *fdDispenser, mounts []specs.Mount) (*fs.Inode, error) {
- // First construct the filesystem from the spec.Root.
- mf := fs.MountSourceFlags{ReadOnly: spec.Root.Readonly || conf.Overlay}
-
- var (
- rootInode *fs.Inode
- err error
- )
-
- fd := fds.remove()
- log.Infof("Mounting root over 9P, ioFD: %d", fd)
- p9FS := mustFindFilesystem("9p")
- opts := p9MountOptions(fd, conf.FileAccess)
- rootInode, err = p9FS.Mount(ctx, rootDevice, mf, strings.Join(opts, ","), nil)
- if err != nil {
- return nil, fmt.Errorf("creating root mount point: %v", err)
- }
-
- // We need to overlay the root on top of a ramfs with stub directories
- // for submount paths. "/dev" "/sys" "/proc" and "/tmp" are always
- // mounted even if they are not in the spec.
- submounts := append(subtargets("/", mounts), "/dev", "/sys", "/proc", "/tmp")
- rootInode, err = addSubmountOverlay(ctx, rootInode, submounts)
- if err != nil {
- return nil, fmt.Errorf("adding submount overlay: %v", err)
- }
-
- if conf.Overlay && !spec.Root.Readonly {
- log.Debugf("Adding overlay on top of root mount")
- // Overlay a tmpfs filesystem on top of the root.
- rootInode, err = addOverlay(ctx, conf, rootInode, "root-overlay-upper", mf)
- if err != nil {
- return nil, err
- }
- }
-
- log.Infof("Mounted %q to %q type root", spec.Root.Path, "/")
- return rootInode, nil
-}
-
-func addOverlay(ctx context.Context, conf *Config, lower *fs.Inode, name string, lowerFlags fs.MountSourceFlags) (*fs.Inode, error) {
- // Upper layer uses the same flags as lower, but it must be read-write.
- lowerFlags.ReadOnly = false
-
- tmpFS := mustFindFilesystem("tmpfs")
- if !fs.IsDir(lower.StableAttr) {
- // Create overlay on top of mount file, e.g. /etc/hostname.
- msrc := fs.NewCachingMountSource(tmpFS, lowerFlags)
- return fs.NewOverlayRootFile(ctx, msrc, lower, lowerFlags)
- }
-
- // Create overlay on top of mount dir.
- upper, err := tmpFS.Mount(ctx, name+"-upper", lowerFlags, "", nil)
- if err != nil {
- return nil, fmt.Errorf("creating tmpfs overlay: %v", err)
- }
- return fs.NewOverlayRoot(ctx, upper, lower, lowerFlags)
-}
-
-// getMountNameAndOptions retrieves the fsName, opts, and useOverlay values
-// used for mounts.
-func getMountNameAndOptions(conf *Config, m specs.Mount, fds *fdDispenser) (string, []string, bool, error) {
- var (
- fsName string
- opts []string
- useOverlay bool
- err error
- )
-
- switch m.Type {
- case devpts, devtmpfs, proc, sysfs:
- fsName = m.Type
- case nonefs:
- fsName = sysfs
- case tmpfs:
- fsName = m.Type
-
- // tmpfs has some extra supported options that we must pass through.
- opts, err = parseAndFilterOptions(m.Options, "mode", "uid", "gid")
-
- case bind:
- fd := fds.remove()
- fsName = "9p"
- // Non-root bind mounts are always shared.
- opts = p9MountOptions(fd, FileAccessShared)
- // If configured, add overlay to all writable mounts.
- useOverlay = conf.Overlay && !mountFlags(m.Options).ReadOnly
-
- default:
- // TODO(nlacasse): Support all the mount types and make this a
- // fatal error. Most applications will "just work" without
- // them, so this is a warning for now.
- // we do not support.
- log.Warningf("ignoring unknown filesystem type %q", m.Type)
- }
- return fsName, opts, useOverlay, err
-}
-
-func mountSubmounts(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *fs.Dirent, mounts []specs.Mount, fds *fdDispenser) error {
- for _, m := range mounts {
- if err := mountSubmount(ctx, conf, mns, root, fds, m, mounts); err != nil {
- return fmt.Errorf("mount submount %q: %v", m.Destination, err)
- }
- }
-
- if err := mountTmp(ctx, conf, mns, root, mounts); err != nil {
- return fmt.Errorf("mount submount %q: %v", "tmp", err)
- }
-
- if !fds.empty() {
- return fmt.Errorf("not all mount points were consumed, remaining: %v", fds)
- }
- return nil
-}
-
-// mountSubmount mounts volumes inside the container's root. Because mounts may
-// be readonly, a lower ramfs overlay is added to create the mount point dir.
-// Another overlay is added with tmpfs on top if Config.Overlay is true.
-// 'm.Destination' must be an absolute path with '..' and symlinks resolved.
-func mountSubmount(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *fs.Dirent, fds *fdDispenser, m specs.Mount, mounts []specs.Mount) error {
- // Map mount type to filesystem name, and parse out the options that we are
- // capable of dealing with.
- fsName, opts, useOverlay, err := getMountNameAndOptions(conf, m, fds)
-
- // Return the error or nil that corresponds to the default case in getMountNameAndOptions.
- if err != nil {
- return err
- }
- if fsName == "" {
- return nil
- }
-
- // All filesystem names should have been mapped to something we know.
- filesystem := mustFindFilesystem(fsName)
-
- mf := mountFlags(m.Options)
- if useOverlay {
- // All writes go to upper, be paranoid and make lower readonly.
- mf.ReadOnly = true
- }
-
- inode, err := filesystem.Mount(ctx, mountDevice(m), mf, strings.Join(opts, ","), nil)
- if err != nil {
- return fmt.Errorf("creating mount with source %q: %v", m.Source, err)
- }
-
- // If there are submounts, we need to overlay the mount on top of a
- // ramfs with stub directories for submount paths.
- submounts := subtargets(m.Destination, mounts)
- if len(submounts) > 0 {
- log.Infof("Adding submount overlay over %q", m.Destination)
- inode, err = addSubmountOverlay(ctx, inode, submounts)
- if err != nil {
- return fmt.Errorf("adding submount overlay: %v", err)
- }
- }
-
- if useOverlay {
- log.Debugf("Adding overlay on top of mount %q", m.Destination)
- inode, err = addOverlay(ctx, conf, inode, m.Type, mf)
- if err != nil {
- return err
- }
- }
-
- maxTraversals := uint(0)
- dirent, err := mns.FindInode(ctx, root, root, m.Destination, &maxTraversals)
- if err != nil {
- return fmt.Errorf("can't find mount destination %q: %v", m.Destination, err)
- }
- defer dirent.DecRef()
- if err := mns.Mount(ctx, dirent, inode); err != nil {
- return fmt.Errorf("mount %q error: %v", m.Destination, err)
- }
-
- log.Infof("Mounted %q to %q type %s", m.Source, m.Destination, m.Type)
- return nil
-}
-
// p9MountOptions creates a slice of options for a p9 mount.
func p9MountOptions(fd int, fa FileAccessType) []string {
opts := []string{
@@ -416,82 +192,6 @@ func mountDevice(m specs.Mount) string {
return "none"
}
-// addRestoreMount adds a mount to the MountSources map used for restoring a
-// checkpointed container.
-func addRestoreMount(conf *Config, renv *fs.RestoreEnvironment, m specs.Mount, fds *fdDispenser) error {
- fsName, opts, useOverlay, err := getMountNameAndOptions(conf, m, fds)
-
- // Return the error or nil that corresponds to the default case in getMountNameAndOptions.
- if err != nil {
- return err
- }
- // TODO(nlacasse): Fix this when we support all the mount types and
- // make this a fatal error.
- if fsName == "" {
- return nil
- }
-
- newMount := fs.MountArgs{
- Dev: mountDevice(m),
- Flags: mountFlags(m.Options),
- DataString: strings.Join(opts, ","),
- }
- if useOverlay {
- newMount.Flags.ReadOnly = true
- }
- renv.MountSources[fsName] = append(renv.MountSources[fsName], newMount)
- log.Infof("Added mount at %q: %+v", fsName, newMount)
- return nil
-}
-
-// createRestoreEnvironment builds a fs.RestoreEnvironment called renv by adding the mounts
-// to the environment.
-func createRestoreEnvironment(spec *specs.Spec, conf *Config, fds *fdDispenser) (*fs.RestoreEnvironment, error) {
- renv := &fs.RestoreEnvironment{
- MountSources: make(map[string][]fs.MountArgs),
- }
-
- // Add root mount.
- fd := fds.remove()
- opts := p9MountOptions(fd, conf.FileAccess)
-
- mf := fs.MountSourceFlags{}
- if spec.Root.Readonly || conf.Overlay {
- mf.ReadOnly = true
- }
-
- rootMount := fs.MountArgs{
- Dev: rootDevice,
- Flags: mf,
- DataString: strings.Join(opts, ","),
- }
- renv.MountSources[rootFsName] = append(renv.MountSources[rootFsName], rootMount)
-
- // Add submounts.
- var tmpMounted bool
- for _, m := range compileMounts(spec) {
- if err := addRestoreMount(conf, renv, m, fds); err != nil {
- return nil, err
- }
- if filepath.Clean(m.Destination) == "/tmp" {
- tmpMounted = true
- }
- }
-
- // TODO(b/67958150): handle '/tmp' properly (see mountTmp()).
- if !tmpMounted {
- tmpMount := specs.Mount{
- Type: tmpfs,
- Destination: "/tmp",
- }
- if err := addRestoreMount(conf, renv, tmpMount, fds); err != nil {
- return nil, err
- }
- }
-
- return renv, nil
-}
-
func mountFlags(opts []string) fs.MountSourceFlags {
mf := fs.MountSourceFlags{}
for _, o := range opts {
@@ -546,22 +246,83 @@ func subtargets(root string, mnts []specs.Mount) []string {
return targets
}
-// setupContainerFS is used to set up the file system and amend the procArgs accordingly.
-// procArgs are passed by reference and the FDMap field is modified. It dups stdioFDs.
-func setupContainerFS(procArgs *kernel.CreateProcessArgs, spec *specs.Spec, conf *Config, stdioFDs, goferFDs []int, console bool, creds *auth.Credentials, ls *limits.LimitSet, k *kernel.Kernel, cid string) error {
- ctx := procArgs.NewContext(k)
-
- // 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.
- fdm, err := createFDMap(ctx, k, ls, console, stdioFDs)
+// setExecutablePath sets the procArgs.Filename by searching the PATH for an
+// executable matching the procArgs.Argv[0].
+func setExecutablePath(ctx context.Context, mns *fs.MountNamespace, procArgs *kernel.CreateProcessArgs) error {
+ paths := fs.GetPath(procArgs.Envv)
+ exe := procArgs.Argv[0]
+ f, err := mns.ResolveExecutablePath(ctx, procArgs.WorkingDirectory, exe, paths)
if err != nil {
- return fmt.Errorf("importing fds: %v", err)
+ return fmt.Errorf("searching for executable %q, cwd: %q, $PATH=%q: %v", exe, procArgs.WorkingDirectory, strings.Join(paths, ":"), err)
+ }
+ procArgs.Filename = f
+ return nil
+}
+
+func adjustDirentCache(k *kernel.Kernel) error {
+ var hl syscall.Rlimit
+ if err := syscall.Getrlimit(syscall.RLIMIT_NOFILE, &hl); err != nil {
+ return fmt.Errorf("getting RLIMIT_NOFILE: %v", err)
+ }
+ if int64(hl.Cur) != syscall.RLIM_INFINITY {
+ newSize := hl.Cur / 2
+ if newSize < gofer.DefaultDirentCacheSize {
+ log.Infof("Setting gofer dirent cache size to %d", newSize)
+ gofer.DefaultDirentCacheSize = newSize
+ k.DirentCacheLimiter = fs.NewDirentCacheLimiter(newSize)
+ }
}
+ return nil
+}
- // CreateProcess takes a reference on FDMap if successful. We
- // won't need ours either way.
- procArgs.FDMap = fdm
+type fdDispenser struct {
+ fds []int
+}
+
+func (f *fdDispenser) remove() int {
+ if f.empty() {
+ panic("fdDispenser out of fds")
+ }
+ rv := f.fds[0]
+ f.fds = f.fds[1:]
+ return rv
+}
+func (f *fdDispenser) empty() bool {
+ return len(f.fds) == 0
+}
+
+type containerMounter struct {
+ // cid is the container ID. May be set to empty for the root container.
+ cid string
+
+ root *specs.Root
+
+ // mounts is the set of submounts for the container. It's a copy from the spec
+ // that may be freely modified without affecting the original spec.
+ mounts []specs.Mount
+
+ // fds is the list of FDs to be dispensed for mounts that require it.
+ fds fdDispenser
+
+ k *kernel.Kernel
+}
+
+func newContainerMounter(spec *specs.Spec, cid string, goferFDs []int, k *kernel.Kernel) *containerMounter {
+ return &containerMounter{
+ cid: cid,
+ root: spec.Root,
+ mounts: compileMounts(spec),
+ fds: fdDispenser{fds: goferFDs},
+ k: k,
+ }
+}
+
+// setupFS is used to set up the file system for containers and amend
+// the procArgs accordingly. This is the main entry point for this rest of
+// functions in this file. procArgs are passed by reference and the FDMap field
+// is modified. It dups stdioFDs.
+func (c *containerMounter) setupFS(ctx context.Context, conf *Config, procArgs *kernel.CreateProcessArgs, creds *auth.Credentials) error {
// Use root user to configure mounts. The current user might not have
// permission to do so.
rootProcArgs := kernel.CreateProcessArgs{
@@ -570,16 +331,19 @@ func setupContainerFS(procArgs *kernel.CreateProcessArgs, spec *specs.Spec, conf
Umask: 0022,
MaxSymlinkTraversals: linux.MaxSymlinkTraversals,
}
- rootCtx := rootProcArgs.NewContext(k)
+ rootCtx := rootProcArgs.NewContext(c.k)
// If this is the root container, we also need to setup the root mount
// namespace.
- mns := k.RootMountNamespace()
+ mns := c.k.RootMountNamespace()
if mns == nil {
// Setup the root container.
- return setupRootContainerFS(ctx, rootCtx, spec, conf, goferFDs, func(mns *fs.MountNamespace) {
- k.SetRootMountNamespace(mns)
- })
+ if err := c.setupRootContainer(ctx, rootCtx, conf, func(mns *fs.MountNamespace) {
+ c.k.SetRootMountNamespace(mns)
+ }); err != nil {
+ return err
+ }
+ return c.checkDispenser()
}
// Setup a child container.
@@ -593,18 +357,17 @@ func setupContainerFS(procArgs *kernel.CreateProcessArgs, spec *specs.Spec, conf
if err != nil {
return fmt.Errorf("couldn't find child container dir %q: %v", ChildContainersDir, err)
}
- if err := contDir.CreateDirectory(ctx, globalRoot, cid, fs.FilePermsFromMode(0755)); err != nil {
- return fmt.Errorf("create directory %q: %v", cid, err)
+ if err := contDir.CreateDirectory(ctx, globalRoot, c.cid, fs.FilePermsFromMode(0755)); err != nil {
+ return fmt.Errorf("create directory %q: %v", c.cid, err)
}
- containerRoot, err := contDir.Walk(ctx, globalRoot, cid)
+ containerRoot, err := contDir.Walk(ctx, globalRoot, c.cid)
if err != nil {
- return fmt.Errorf("walk to %q failed: %v", cid, err)
+ return fmt.Errorf("walk to %q failed: %v", c.cid, err)
}
defer containerRoot.DecRef()
// Create the container's root filesystem mount.
- fds := &fdDispenser{fds: goferFDs}
- rootInode, err := createRootMount(rootCtx, spec, conf, fds, nil)
+ rootInode, err := c.createRootMount(rootCtx, conf)
if err != nil {
return fmt.Errorf("creating filesystem for container: %v", err)
}
@@ -614,39 +377,32 @@ func setupContainerFS(procArgs *kernel.CreateProcessArgs, spec *specs.Spec, conf
return fmt.Errorf("mount container root: %v", err)
}
- // We have to re-walk to the dirent to find the mounted
- // directory. The old dirent is invalid at this point.
- containerRoot, err = contDir.Walk(ctx, globalRoot, cid)
+ // We have to re-walk to the dirent to find the mounted directory. The old
+ // dirent is invalid at this point.
+ containerRoot, err = contDir.Walk(ctx, globalRoot, c.cid)
if err != nil {
- return fmt.Errorf("find container mount point %q: %v", cid, err)
+ return fmt.Errorf("find container mount point %q: %v", c.cid, err)
}
cu := specutils.MakeCleanup(func() { containerRoot.DecRef() })
defer cu.Clean()
- log.Infof("Mounted child's root fs to %q", filepath.Join(ChildContainersDir, cid))
+ log.Infof("Mounted child's root fs to %q", filepath.Join(ChildContainersDir, c.cid))
// Set process root here, so 'rootCtx.Value(CtxRoot)' will return it.
procArgs.Root = containerRoot
// Mount all submounts.
- mounts := compileMounts(spec)
- if err := mountSubmounts(rootCtx, conf, mns, containerRoot, mounts, fds); err != nil {
+ if err := c.mountSubmounts(rootCtx, conf, mns, containerRoot); err != nil {
return err
}
cu.Release()
- return nil
+ return c.checkDispenser()
}
-// setExecutablePath sets the procArgs.Filename by searching the PATH for an
-// executable matching the procArgs.Argv[0].
-func setExecutablePath(ctx context.Context, mns *fs.MountNamespace, procArgs *kernel.CreateProcessArgs) error {
- paths := fs.GetPath(procArgs.Envv)
- exe := procArgs.Argv[0]
- f, err := mns.ResolveExecutablePath(ctx, procArgs.WorkingDirectory, exe, paths)
- if err != nil {
- return fmt.Errorf("searching for executable %q, cwd: %q, $PATH=%q: %v", exe, procArgs.WorkingDirectory, strings.Join(paths, ":"), err)
+func (c *containerMounter) checkDispenser() error {
+ if !c.fds.empty() {
+ return fmt.Errorf("not all gofer FDs were consumed, remaining: %v", c.fds)
}
- procArgs.Filename = f
return nil
}
@@ -715,6 +471,261 @@ func destroyContainerFS(ctx context.Context, cid string, k *kernel.Kernel) error
return nil
}
+// setupRootContainer creates a mount namespace containing the root filesystem
+// and all mounts. 'rootCtx' is used to walk directories to find mount points.
+// 'setMountNS' is called after namespace is created. It must set the mount NS
+// to 'rootCtx'.
+func (c *containerMounter) setupRootContainer(userCtx context.Context, rootCtx context.Context, conf *Config, setMountNS func(*fs.MountNamespace)) error {
+ // Create a tmpfs mount where we create and mount a root filesystem for
+ // each child container.
+ c.mounts = append(c.mounts, specs.Mount{
+ Type: tmpfs,
+ Destination: ChildContainersDir,
+ })
+
+ rootInode, err := c.createRootMount(rootCtx, conf)
+ if err != nil {
+ return fmt.Errorf("creating root mount: %v", err)
+ }
+ mns, err := fs.NewMountNamespace(userCtx, rootInode)
+ if err != nil {
+ return fmt.Errorf("creating root mount namespace: %v", err)
+ }
+ setMountNS(mns)
+
+ root := mns.Root()
+ defer root.DecRef()
+ return c.mountSubmounts(rootCtx, conf, mns, root)
+}
+
+// createRootMount creates the root filesystem.
+func (c *containerMounter) createRootMount(ctx context.Context, conf *Config) (*fs.Inode, error) {
+ // First construct the filesystem from the spec.Root.
+ mf := fs.MountSourceFlags{ReadOnly: c.root.Readonly || conf.Overlay}
+
+ var (
+ rootInode *fs.Inode
+ err error
+ )
+
+ fd := c.fds.remove()
+ log.Infof("Mounting root over 9P, ioFD: %d", fd)
+ p9FS := mustFindFilesystem("9p")
+ opts := p9MountOptions(fd, conf.FileAccess)
+ rootInode, err = p9FS.Mount(ctx, rootDevice, mf, strings.Join(opts, ","), nil)
+ if err != nil {
+ return nil, fmt.Errorf("creating root mount point: %v", err)
+ }
+
+ // We need to overlay the root on top of a ramfs with stub directories
+ // for submount paths. "/dev" "/sys" "/proc" and "/tmp" are always
+ // mounted even if they are not in the spec.
+ submounts := append(subtargets("/", c.mounts), "/dev", "/sys", "/proc", "/tmp")
+ rootInode, err = addSubmountOverlay(ctx, rootInode, submounts)
+ if err != nil {
+ return nil, fmt.Errorf("adding submount overlay: %v", err)
+ }
+
+ if conf.Overlay && !c.root.Readonly {
+ log.Debugf("Adding overlay on top of root mount")
+ // Overlay a tmpfs filesystem on top of the root.
+ rootInode, err = addOverlay(ctx, conf, rootInode, "root-overlay-upper", mf)
+ if err != nil {
+ return nil, err
+ }
+ }
+
+ log.Infof("Mounted %q to %q type root", c.root.Path, "/")
+ return rootInode, nil
+}
+
+// getMountNameAndOptions retrieves the fsName, opts, and useOverlay values
+// used for mounts.
+func (c *containerMounter) getMountNameAndOptions(conf *Config, m specs.Mount) (string, []string, bool, error) {
+ var (
+ fsName string
+ opts []string
+ useOverlay bool
+ err error
+ )
+
+ switch m.Type {
+ case devpts, devtmpfs, proc, sysfs:
+ fsName = m.Type
+ case nonefs:
+ fsName = sysfs
+ case tmpfs:
+ fsName = m.Type
+
+ // tmpfs has some extra supported options that we must pass through.
+ opts, err = parseAndFilterOptions(m.Options, "mode", "uid", "gid")
+
+ case bind:
+ fd := c.fds.remove()
+ fsName = "9p"
+ // Non-root bind mounts are always shared.
+ opts = p9MountOptions(fd, FileAccessShared)
+ // If configured, add overlay to all writable mounts.
+ useOverlay = conf.Overlay && !mountFlags(m.Options).ReadOnly
+
+ default:
+ // TODO(nlacasse): Support all the mount types and make this a fatal error.
+ // Most applications will "just work" without them, so this is a warning
+ // for now.
+ log.Warningf("ignoring unknown filesystem type %q", m.Type)
+ }
+ return fsName, opts, useOverlay, err
+}
+
+func (c *containerMounter) mountSubmounts(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *fs.Dirent) error {
+ for _, m := range c.mounts {
+ if err := c.mountSubmount(ctx, conf, mns, root, m); err != nil {
+ return fmt.Errorf("mount submount %q: %v", m.Destination, err)
+ }
+ }
+
+ if err := c.mountTmp(ctx, conf, mns, root); err != nil {
+ return fmt.Errorf("mount submount %q: %v", "tmp", err)
+ }
+ return nil
+}
+
+// mountSubmount mounts volumes inside the container's root. Because mounts may
+// be readonly, a lower ramfs overlay is added to create the mount point dir.
+// Another overlay is added with tmpfs on top if Config.Overlay is true.
+// 'm.Destination' must be an absolute path with '..' and symlinks resolved.
+func (c *containerMounter) mountSubmount(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *fs.Dirent, m specs.Mount) error {
+ // Map mount type to filesystem name, and parse out the options that we are
+ // capable of dealing with.
+ fsName, opts, useOverlay, err := c.getMountNameAndOptions(conf, m)
+ if err != nil {
+ return err
+ }
+ if fsName == "" {
+ // Filesystem is not supported (e.g. cgroup), just skip it.
+ return nil
+ }
+
+ // All filesystem names should have been mapped to something we know.
+ filesystem := mustFindFilesystem(fsName)
+
+ mf := mountFlags(m.Options)
+ if useOverlay {
+ // All writes go to upper, be paranoid and make lower readonly.
+ mf.ReadOnly = true
+ }
+
+ inode, err := filesystem.Mount(ctx, mountDevice(m), mf, strings.Join(opts, ","), nil)
+ if err != nil {
+ return fmt.Errorf("creating mount with source %q: %v", m.Source, err)
+ }
+
+ // If there are submounts, we need to overlay the mount on top of a ramfs
+ // with stub directories for submount paths.
+ submounts := subtargets(m.Destination, c.mounts)
+ if len(submounts) > 0 {
+ log.Infof("Adding submount overlay over %q", m.Destination)
+ inode, err = addSubmountOverlay(ctx, inode, submounts)
+ if err != nil {
+ return fmt.Errorf("adding submount overlay: %v", err)
+ }
+ }
+
+ if useOverlay {
+ log.Debugf("Adding overlay on top of mount %q", m.Destination)
+ inode, err = addOverlay(ctx, conf, inode, m.Type, mf)
+ if err != nil {
+ return err
+ }
+ }
+
+ maxTraversals := uint(0)
+ dirent, err := mns.FindInode(ctx, root, root, m.Destination, &maxTraversals)
+ if err != nil {
+ return fmt.Errorf("can't find mount destination %q: %v", m.Destination, err)
+ }
+ defer dirent.DecRef()
+ if err := mns.Mount(ctx, dirent, inode); err != nil {
+ return fmt.Errorf("mount %q error: %v", m.Destination, err)
+ }
+
+ log.Infof("Mounted %q to %q type %s", m.Source, m.Destination, m.Type)
+ return nil
+}
+
+// addRestoreMount adds a mount to the MountSources map used for restoring a
+// checkpointed container.
+func (c *containerMounter) addRestoreMount(conf *Config, renv *fs.RestoreEnvironment, m specs.Mount) error {
+ fsName, opts, useOverlay, err := c.getMountNameAndOptions(conf, m)
+ if err != nil {
+ return err
+ }
+ if fsName == "" {
+ // Filesystem is not supported (e.g. cgroup), just skip it.
+ return nil
+ }
+
+ newMount := fs.MountArgs{
+ Dev: mountDevice(m),
+ Flags: mountFlags(m.Options),
+ DataString: strings.Join(opts, ","),
+ }
+ if useOverlay {
+ newMount.Flags.ReadOnly = true
+ }
+ renv.MountSources[fsName] = append(renv.MountSources[fsName], newMount)
+ log.Infof("Added mount at %q: %+v", fsName, newMount)
+ return nil
+}
+
+// createRestoreEnvironment builds a fs.RestoreEnvironment called renv by adding the mounts
+// to the environment.
+func (c *containerMounter) createRestoreEnvironment(conf *Config) (*fs.RestoreEnvironment, error) {
+ renv := &fs.RestoreEnvironment{
+ MountSources: make(map[string][]fs.MountArgs),
+ }
+
+ // Add root mount.
+ fd := c.fds.remove()
+ opts := p9MountOptions(fd, conf.FileAccess)
+
+ mf := fs.MountSourceFlags{}
+ if c.root.Readonly || conf.Overlay {
+ mf.ReadOnly = true
+ }
+
+ rootMount := fs.MountArgs{
+ Dev: rootDevice,
+ Flags: mf,
+ DataString: strings.Join(opts, ","),
+ }
+ renv.MountSources[rootFsName] = append(renv.MountSources[rootFsName], rootMount)
+
+ // Add submounts.
+ var tmpMounted bool
+ for _, m := range c.mounts {
+ if err := c.addRestoreMount(conf, renv, m); err != nil {
+ return nil, err
+ }
+ if filepath.Clean(m.Destination) == "/tmp" {
+ tmpMounted = true
+ }
+ }
+
+ // TODO(b/67958150): handle '/tmp' properly (see mountTmp()).
+ if !tmpMounted {
+ tmpMount := specs.Mount{
+ Type: tmpfs,
+ Destination: "/tmp",
+ }
+ if err := c.addRestoreMount(conf, renv, tmpMount); err != nil {
+ return nil, err
+ }
+ }
+
+ return renv, nil
+}
+
// mountTmp mounts an internal tmpfs at '/tmp' if it's safe to do so.
// Technically we don't have to mount tmpfs at /tmp, as we could just rely on
// the host /tmp, but this is a nice optimization, and fixes some apps that call
@@ -724,8 +735,8 @@ func destroyContainerFS(ctx context.Context, cid string, k *kernel.Kernel) error
//
// Note that when there are submounts inside of '/tmp', directories for the
// mount points must be present, making '/tmp' not empty anymore.
-func mountTmp(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *fs.Dirent, mounts []specs.Mount) error {
- for _, m := range mounts {
+func (c *containerMounter) mountTmp(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *fs.Dirent) error {
+ for _, m := range c.mounts {
if filepath.Clean(m.Destination) == "/tmp" {
log.Debugf("Explict %q mount found, skipping internal tmpfs, mount: %+v", "/tmp", m)
return nil
@@ -766,7 +777,7 @@ func mountTmp(ctx context.Context, conf *Config, mns *fs.MountNamespace, root *f
// another user. This is normally done for /tmp.
Options: []string{"mode=1777"},
}
- return mountSubmount(ctx, conf, mns, root, nil, tmpMount, mounts)
+ return c.mountSubmount(ctx, conf, mns, root, tmpMount)
default:
return err
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index 6ac6b94dd..a997776f8 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -288,7 +288,7 @@ func New(args Args) (*Loader, error) {
}
// Create a watchdog.
- watchdog := watchdog.New(k, watchdog.DefaultTimeout, args.Conf.WatchdogAction)
+ dog := watchdog.New(k, watchdog.DefaultTimeout, args.Conf.WatchdogAction)
procArgs, err := newProcess(args.ID, args.Spec, creds, k)
if err != nil {
@@ -304,7 +304,7 @@ func New(args Args) (*Loader, error) {
k: k,
conf: args.Conf,
console: args.Console,
- watchdog: watchdog,
+ watchdog: dog,
spec: args.Spec,
goferFDs: args.GoferFDs,
stdioFDs: args.StdioFDs,
@@ -432,7 +432,7 @@ func createMemoryFile() (*pgalloc.MemoryFile, error) {
return mf, nil
}
-// Run runs the root container..
+// Run runs the root container.
func (l *Loader) Run() error {
err := l.run()
l.ctrl.manager.startResultChan <- err
@@ -486,17 +486,21 @@ 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.
if !l.restore {
- if err := setupContainerFS(
- &l.rootProcArgs,
- l.spec,
- l.conf,
- l.stdioFDs,
- l.goferFDs,
- l.console,
- l.rootProcArgs.Credentials,
- l.rootProcArgs.Limits,
- l.k,
- "" /* CID, which isn't needed for the root container */); err != nil {
+ // 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)
+ fdm, err := createFDMap(ctx, l.rootProcArgs.Limits, 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.FDMap = fdm
+
+ // cid for root container can be empty. Only subcontainers need it to set
+ // the mount location.
+ mntr := newContainerMounter(l.spec, "", l.goferFDs, l.k)
+ if err := mntr.setupFS(ctx, l.conf, &l.rootProcArgs, l.rootProcArgs.Credentials); err != nil {
return err
}
@@ -552,7 +556,7 @@ func (l *Loader) createContainer(cid string) error {
// startContainer starts a child container. It returns the thread group ID of
// the newly created process. Caller owns 'files' and may close them after
// this method returns.
-func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config, cid string, files []*os.File) error {
+func (l *Loader) startContainer(spec *specs.Spec, conf *Config, cid string, files []*os.File) error {
// Create capabilities.
caps, err := specutils.Capabilities(conf.EnableRaw, spec.Process.Capabilities)
if err != nil {
@@ -596,6 +600,16 @@ func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config
stdioFDs = append(stdioFDs, int(f.Fd()))
}
+ // Create the FD map, which will set stdin, stdout, and stderr.
+ ctx := procArgs.NewContext(l.k)
+ fdm, err := createFDMap(ctx, procArgs.Limits, false, 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.
+ procArgs.FDMap = fdm
+
// Can't take ownership away from os.File. dup them to get a new FDs.
var goferFDs []int
for _, f := range files[3:] {
@@ -606,22 +620,12 @@ func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config
goferFDs = append(goferFDs, fd)
}
- if err := setupContainerFS(
- &procArgs,
- spec,
- conf,
- stdioFDs,
- goferFDs,
- false,
- creds,
- procArgs.Limits,
- k,
- cid); err != nil {
+ mntr := newContainerMounter(spec, cid, goferFDs, l.k)
+ if err := mntr.setupFS(ctx, conf, &procArgs, creds); err != nil {
return fmt.Errorf("configuring container FS: %v", err)
}
- ctx := procArgs.NewContext(l.k)
- mns := k.RootMountNamespace()
+ mns := l.k.RootMountNamespace()
if err := setExecutablePath(ctx, mns, &procArgs); err != nil {
return fmt.Errorf("setting executable path for %+v: %v", procArgs, err)
}
@@ -724,7 +728,7 @@ func (l *Loader) waitContainer(cid string, waitStatus *uint32) error {
return nil
}
-func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, clearStatus bool, waitStatus *uint32) error {
+func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, waitStatus *uint32) error {
if tgid <= 0 {
return fmt.Errorf("PID (%d) must be positive", tgid)
}
@@ -736,13 +740,10 @@ func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, clearStatus bool, wai
ws := l.wait(execTG)
*waitStatus = ws
- // Remove tg from the cache if caller requested it.
- if clearStatus {
- l.mu.Lock()
- delete(l.processes, eid)
- log.Debugf("updated processes (removal): %v", l.processes)
- l.mu.Unlock()
- }
+ l.mu.Lock()
+ delete(l.processes, eid)
+ log.Debugf("updated processes (removal): %v", l.processes)
+ l.mu.Unlock()
return nil
}