summaryrefslogtreecommitdiffhomepage
path: root/runsc/boot
diff options
context:
space:
mode:
Diffstat (limited to 'runsc/boot')
-rw-r--r--runsc/boot/BUILD4
-rw-r--r--runsc/boot/config.go13
-rw-r--r--runsc/boot/controller.go27
-rw-r--r--runsc/boot/fds.go3
-rw-r--r--runsc/boot/filter/config.go4
-rw-r--r--runsc/boot/fs.go986
-rw-r--r--runsc/boot/fs_test.go193
-rw-r--r--runsc/boot/loader.go131
-rw-r--r--runsc/boot/loader_test.go9
-rw-r--r--runsc/boot/network.go41
-rw-r--r--runsc/boot/pprof.go18
11 files changed, 988 insertions, 441 deletions
diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD
index df9907e52..744f852a1 100644
--- a/runsc/boot/BUILD
+++ b/runsc/boot/BUILD
@@ -16,6 +16,7 @@ go_library(
"limits.go",
"loader.go",
"network.go",
+ "pprof.go",
"strace.go",
],
importpath = "gvisor.googlesource.com/gvisor/runsc/boot",
@@ -30,6 +31,7 @@ go_library(
"//pkg/cpuid",
"//pkg/eventchannel",
"//pkg/log",
+ "//pkg/memutil",
"//pkg/rand",
"//pkg/sentry/arch",
"//pkg/sentry/arch:registers_go_proto",
@@ -51,7 +53,6 @@ go_library(
"//pkg/sentry/kernel/kdefs",
"//pkg/sentry/limits",
"//pkg/sentry/loader",
- "//pkg/sentry/memutil",
"//pkg/sentry/pgalloc",
"//pkg/sentry/platform",
"//pkg/sentry/platform/kvm",
@@ -94,6 +95,7 @@ go_test(
size = "small",
srcs = [
"compat_test.go",
+ "fs_test.go",
"loader_test.go",
],
embed = [":boot"],
diff --git a/runsc/boot/config.go b/runsc/boot/config.go
index 15f624f9b..6112b6c0a 100644
--- a/runsc/boot/config.go
+++ b/runsc/boot/config.go
@@ -221,6 +221,17 @@ type Config struct {
// user, and without chrooting the sandbox process. This can be
// necessary in test environments that have limited capabilities.
TestOnlyAllowRunAsCurrentUserWithoutChroot bool
+
+ // NumNetworkChannels controls the number of AF_PACKET sockets that map
+ // to the same underlying network device. This allows netstack to better
+ // scale for high throughput use cases.
+ NumNetworkChannels int
+
+ // Rootless allows the sandbox to be started with a user that is not root.
+ // Defense is depth measures are weaker with rootless. Specifically, the
+ // sandbox and Gofer process run as root inside a user namespace with root
+ // mapped to the caller's user.
+ Rootless bool
}
// ToFlags returns a slice of flags that correspond to the given Config.
@@ -244,6 +255,8 @@ func (c *Config) ToFlags() []string {
"--panic-signal=" + strconv.Itoa(c.PanicSignal),
"--profile=" + strconv.FormatBool(c.ProfileEnable),
"--net-raw=" + strconv.FormatBool(c.EnableRaw),
+ "--num-network-channels=" + strconv.Itoa(c.NumNetworkChannels),
+ "--rootless=" + strconv.FormatBool(c.Rootless),
}
if c.TestOnlyAllowRunAsCurrentUserWithoutChroot {
// Only include if set since it is never to be used by users.
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go
index 72ab9ef86..26765cc46 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, cm.l.mountHints)
+ renv, err := mntr.createRestoreEnvironment(cm.l.conf)
if err != nil {
return fmt.Errorf("creating RestoreEnvironment: %v", err)
}
@@ -359,6 +359,17 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error {
return fmt.Errorf("file cannot be empty")
}
+ if cm.l.conf.ProfileEnable {
+ // initializePProf opens /proc/self/maps, so has to be
+ // called before installing seccomp filters.
+ initializePProf()
+ }
+
+ // Seccomp filters have to be applied before parsing the state file.
+ if err := cm.l.installSeccompFilters(); err != nil {
+ return err
+ }
+
// Load the state.
loadOpts := state.LoadOpts{Source: specFile}
if err := loadOpts.Load(k, networkStack); err != nil {
@@ -369,11 +380,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 +431,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/filter/config.go b/runsc/boot/filter/config.go
index 652da1cef..ef2dbfad2 100644
--- a/runsc/boot/filter/config.go
+++ b/runsc/boot/filter/config.go
@@ -246,6 +246,10 @@ var allowedSyscalls = seccomp.SyscallRules{
},
syscall.SYS_SETITIMER: {},
syscall.SYS_SHUTDOWN: []seccomp.Rule{
+ // Used by fs/host to shutdown host sockets.
+ {seccomp.AllowAny{}, seccomp.AllowValue(syscall.SHUT_RD)},
+ {seccomp.AllowAny{}, seccomp.AllowValue(syscall.SHUT_WR)},
+ // Used by unet to shutdown connections.
{seccomp.AllowAny{}, seccomp.AllowValue(syscall.SHUT_RDWR)},
},
syscall.SYS_SIGALTSTACK: {},
diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go
index 4b1557b9a..2fa0725d1 100644
--- a/runsc/boot/fs.go
+++ b/runsc/boot/fs.go
@@ -18,6 +18,7 @@ import (
"fmt"
"path"
"path/filepath"
+ "sort"
"strconv"
"strings"
"syscall"
@@ -29,9 +30,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 +38,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"
)
@@ -51,6 +51,9 @@ const (
// Device name for root mount.
rootDevice = "9pfs-/"
+ // MountPrefix is the annotation prefix for mount hints.
+ MountPrefix = "gvisor.dev/spec/mount"
+
// ChildContainersDir is the directory where child container root
// filesystems are mounted.
ChildContainersDir = "/__runsc_containers__"
@@ -65,67 +68,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 +144,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 +196,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 +250,254 @@ 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 shareType int
+
+const (
+ invalid shareType = iota
+
+ // container shareType indicates that the mount is used by a single container.
+ container
+
+ // pod shareType indicates that the mount is used by more than one container
+ // inside the pod.
+ pod
+
+ // shared shareType indicates that the mount can also be shared with a process
+ // outside the pod, e.g. NFS.
+ shared
+)
+
+func parseShare(val string) (shareType, error) {
+ switch val {
+ case "container":
+ return container, nil
+ case "pod":
+ return pod, nil
+ case "shared":
+ return shared, nil
+ default:
+ return 0, fmt.Errorf("invalid share value %q", val)
+ }
+}
+
+func (s shareType) String() string {
+ switch s {
+ case invalid:
+ return "invalid"
+ case container:
+ return "container"
+ case pod:
+ return "pod"
+ case shared:
+ return "shared"
+ default:
+ return fmt.Sprintf("invalid share value %d", s)
+ }
+}
+
+// mountHint represents extra information about mounts that are provided via
+// annotations. They can override mount type, and provide sharing information
+// so that mounts can be correctly shared inside the pod.
+type mountHint struct {
+ name string
+ share shareType
+ mount specs.Mount
+
+ // root is the inode where the volume is mounted. For mounts with 'pod' share
+ // the volume is mounted once and then bind mounted inside the containers.
+ root *fs.Inode
+}
+
+func (m *mountHint) setField(key, val string) error {
+ switch key {
+ case "source":
+ if len(val) == 0 {
+ return fmt.Errorf("source cannot be empty")
+ }
+ m.mount.Source = val
+ case "type":
+ return m.setType(val)
+ case "share":
+ share, err := parseShare(val)
+ if err != nil {
+ return err
+ }
+ m.share = share
+ case "options":
+ return m.setOptions(val)
+ default:
+ return fmt.Errorf("invalid mount annotation: %s=%s", key, val)
+ }
+ return nil
+}
+
+func (m *mountHint) setType(val string) error {
+ switch val {
+ case "tmpfs", "bind":
+ m.mount.Type = val
+ default:
+ return fmt.Errorf("invalid type %q", val)
+ }
+ return nil
+}
+
+func (m *mountHint) setOptions(val string) error {
+ opts := strings.Split(val, ",")
+ if err := specutils.ValidateMountOptions(opts); err != nil {
+ return err
+ }
+ // Sort options so it can be compared with container mount options later on.
+ sort.Strings(opts)
+ m.mount.Options = opts
+ return nil
+}
+
+func (m *mountHint) isSupported() bool {
+ return m.mount.Type == tmpfs && m.share == pod
+}
+
+// podMountHints contains a collection of mountHints for the pod.
+type podMountHints struct {
+ mounts map[string]*mountHint
+}
+
+func newPodMountHints(spec *specs.Spec) (*podMountHints, error) {
+ mnts := make(map[string]*mountHint)
+ for k, v := range spec.Annotations {
+ // Look for 'gvisor.dev/spec/mount' annotations and parse them.
+ if strings.HasPrefix(k, MountPrefix) {
+ parts := strings.Split(k, "/")
+ if len(parts) != 5 {
+ return nil, fmt.Errorf("invalid mount annotation: %s=%s", k, v)
+ }
+ name := parts[3]
+ if len(name) == 0 || path.Clean(name) != name {
+ return nil, fmt.Errorf("invalid mount name: %s", name)
+ }
+ mnt := mnts[name]
+ if mnt == nil {
+ mnt = &mountHint{name: name}
+ mnts[name] = mnt
+ }
+ if err := mnt.setField(parts[4], v); err != nil {
+ return nil, err
+ }
+ }
+ }
+
+ // Validate all hints after done parsing.
+ for name, m := range mnts {
+ log.Infof("Mount annotation found, name: %s, source: %q, type: %s, share: %v", name, m.mount.Source, m.mount.Type, m.share)
+ if m.share == invalid {
+ return nil, fmt.Errorf("share field for %q has not been set", m.name)
+ }
+ if len(m.mount.Source) == 0 {
+ return nil, fmt.Errorf("source field for %q has not been set", m.name)
+ }
+ if len(m.mount.Type) == 0 {
+ return nil, fmt.Errorf("type field for %q has not been set", m.name)
+ }
+
+ // Check for duplicate mount sources.
+ for name2, m2 := range mnts {
+ if name != name2 && m.mount.Source == m2.mount.Source {
+ return nil, fmt.Errorf("mounts %q and %q have the same mount source %q", m.name, m2.name, m.mount.Source)
+ }
+ }
+ }
+
+ return &podMountHints{mounts: mnts}, nil
+}
+
+func (p *podMountHints) findMount(mount specs.Mount) *mountHint {
+ for _, m := range p.mounts {
+ if m.mount.Source == mount.Source {
+ return m
+ }
+ }
+ return nil
+}
+
+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
+
+ hints *podMountHints
+}
+
+func newContainerMounter(spec *specs.Spec, cid string, goferFDs []int, k *kernel.Kernel, hints *podMountHints) *containerMounter {
+ return &containerMounter{
+ cid: cid,
+ root: spec.Root,
+ mounts: compileMounts(spec),
+ fds: fdDispenser{fds: goferFDs},
+ k: k,
+ hints: hints,
+ }
+}
+
+// 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 +506,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 +532,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 +552,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,17 +646,354 @@ 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 {
+ for _, hint := range c.hints.mounts {
+ log.Infof("Mounting master of shared mount %q from %q type %q", hint.name, hint.mount.Source, hint.mount.Type)
+ inode, err := c.mountSharedMaster(rootCtx, conf, hint)
+ if err != nil {
+ return fmt.Errorf("mounting shared master %q: %v", hint.name, err)
+ }
+ hint.root = inode
+ }
+
+ // 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)
+}
+
+// mountSharedMaster mounts the master of a volume that is shared among
+// containers in a pod. It returns the root mount's inode.
+func (c *containerMounter) mountSharedMaster(ctx context.Context, conf *Config, hint *mountHint) (*fs.Inode, 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, hint.mount)
+ if err != nil {
+ return nil, err
+ }
+ if len(fsName) == 0 {
+ return nil, fmt.Errorf("mount type not supported %q", hint.mount.Type)
+ }
+
+ // Mount with revalidate because it's shared among containers.
+ opts = append(opts, "cache=revalidate")
+
+ // All filesystem names should have been mapped to something we know.
+ filesystem := mustFindFilesystem(fsName)
+
+ mf := mountFlags(hint.mount.Options)
+ if useOverlay {
+ // All writes go to upper, be paranoid and make lower readonly.
+ mf.ReadOnly = true
+ }
+
+ inode, err := filesystem.Mount(ctx, mountDevice(hint.mount), mf, strings.Join(opts, ","), nil)
+ if err != nil {
+ return nil, fmt.Errorf("creating mount %q: %v", hint.name, err)
+ }
+
+ if useOverlay {
+ log.Debugf("Adding overlay on top of shared mount %q", hint.name)
+ inode, err = addOverlay(ctx, conf, inode, hint.mount.Type, mf)
+ if err != nil {
+ return nil, err
+ }
+ }
+
+ return inode, nil
+}
+
+// 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}
+
+ 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 hint := c.hints.findMount(m); hint != nil && hint.isSupported() {
+ if err := c.mountSharedSubmount(ctx, mns, root, m, hint); err != nil {
+ return fmt.Errorf("mount shared mount %q to %q: %v", hint.name, m.Destination, err)
+ }
+ } else {
+ 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
+}
+
+// mountSharedSubmount binds mount to a previously mounted volume that is shared
+// among containers in the same pod.
+func (c *containerMounter) mountSharedSubmount(ctx context.Context, mns *fs.MountNamespace, root *fs.Dirent, mount specs.Mount, source *mountHint) error {
+ // For now enforce that all options are the same. Once bind mount is properly
+ // supported, then we should ensure the master is less restrictive than the
+ // container, e.g. master can be 'rw' while container mounts as 'ro'.
+ if len(mount.Options) != len(source.mount.Options) {
+ return fmt.Errorf("mount options in annotations differ from container mount, annotation: %s, mount: %s", source.mount.Options, mount.Options)
+ }
+ sort.Strings(mount.Options)
+ for i, opt := range mount.Options {
+ if opt != source.mount.Options[i] {
+ return fmt.Errorf("mount options in annotations differ from container mount, annotation: %s, mount: %s", source.mount.Options, mount.Options)
+ }
+ }
+
+ maxTraversals := uint(0)
+ target, err := mns.FindInode(ctx, root, root, mount.Destination, &maxTraversals)
+ if err != nil {
+ return fmt.Errorf("can't find mount destination %q: %v", mount.Destination, err)
+ }
+ defer target.DecRef()
+
+ if err := mns.Mount(ctx, target, source.root); err != nil {
+ return fmt.Errorf("bind mount %q error: %v", mount.Destination, err)
+ }
+
+ log.Infof("Mounted %q type shared bind to %q", mount.Destination, source.name)
+ 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
// mknod in /tmp. It's unsafe to mount tmpfs if:
-// 1. /tmp is mounted explictly: we should not override user's wish
+// 1. /tmp is mounted explicitly: we should not override user's wish
// 2. /tmp is not empty: mounting tmpfs would hide existing files in /tmp
//
// 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 +1034,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/fs_test.go b/runsc/boot/fs_test.go
new file mode 100644
index 000000000..49ab34b33
--- /dev/null
+++ b/runsc/boot/fs_test.go
@@ -0,0 +1,193 @@
+// Copyright 2019 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package boot
+
+import (
+ "path"
+ "reflect"
+ "strings"
+ "testing"
+
+ specs "github.com/opencontainers/runtime-spec/specs-go"
+)
+
+func TestPodMountHintsHappy(t *testing.T) {
+ spec := &specs.Spec{
+ Annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+
+ path.Join(MountPrefix, "mount2", "source"): "bar",
+ path.Join(MountPrefix, "mount2", "type"): "bind",
+ path.Join(MountPrefix, "mount2", "share"): "container",
+ path.Join(MountPrefix, "mount2", "options"): "rw,private",
+ },
+ }
+ podHints, err := newPodMountHints(spec)
+ if err != nil {
+ t.Errorf("newPodMountHints failed: %v", err)
+ }
+
+ // Check that fields were set correctly.
+ mount1 := podHints.mounts["mount1"]
+ if want := "mount1"; want != mount1.name {
+ t.Errorf("mount1 name, want: %q, got: %q", want, mount1.name)
+ }
+ if want := "foo"; want != mount1.mount.Source {
+ t.Errorf("mount1 source, want: %q, got: %q", want, mount1.mount.Source)
+ }
+ if want := "tmpfs"; want != mount1.mount.Type {
+ t.Errorf("mount1 type, want: %q, got: %q", want, mount1.mount.Type)
+ }
+ if want := pod; want != mount1.share {
+ t.Errorf("mount1 type, want: %q, got: %q", want, mount1.share)
+ }
+ if want := []string(nil); !reflect.DeepEqual(want, mount1.mount.Options) {
+ t.Errorf("mount1 type, want: %q, got: %q", want, mount1.mount.Options)
+ }
+
+ mount2 := podHints.mounts["mount2"]
+ if want := "mount2"; want != mount2.name {
+ t.Errorf("mount2 name, want: %q, got: %q", want, mount2.name)
+ }
+ if want := "bar"; want != mount2.mount.Source {
+ t.Errorf("mount2 source, want: %q, got: %q", want, mount2.mount.Source)
+ }
+ if want := "bind"; want != mount2.mount.Type {
+ t.Errorf("mount2 type, want: %q, got: %q", want, mount2.mount.Type)
+ }
+ if want := container; want != mount2.share {
+ t.Errorf("mount2 type, want: %q, got: %q", want, mount2.share)
+ }
+ if want := []string{"private", "rw"}; !reflect.DeepEqual(want, mount2.mount.Options) {
+ t.Errorf("mount2 type, want: %q, got: %q", want, mount2.mount.Options)
+ }
+}
+
+func TestPodMountHintsErrors(t *testing.T) {
+ for _, tst := range []struct {
+ name string
+ annotations map[string]string
+ error string
+ }{
+ {
+ name: "too short",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1"): "foo",
+ },
+ error: "invalid mount annotation",
+ },
+ {
+ name: "no name",
+ annotations: map[string]string{
+ MountPrefix + "//source": "foo",
+ },
+ error: "invalid mount name",
+ },
+ {
+ name: "missing source",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+ },
+ error: "source field",
+ },
+ {
+ name: "missing type",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+ },
+ error: "type field",
+ },
+ {
+ name: "missing share",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ },
+ error: "share field",
+ },
+ {
+ name: "invalid field name",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "invalid"): "foo",
+ },
+ error: "invalid mount annotation",
+ },
+ {
+ name: "invalid source",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "",
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+ },
+ error: "source cannot be empty",
+ },
+ {
+ name: "invalid type",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "type"): "invalid-type",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+ },
+ error: "invalid type",
+ },
+ {
+ name: "invalid share",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ path.Join(MountPrefix, "mount1", "share"): "invalid-share",
+ },
+ error: "invalid share",
+ },
+ {
+ name: "invalid options",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+ path.Join(MountPrefix, "mount1", "options"): "invalid-option",
+ },
+ error: "unknown mount option",
+ },
+ {
+ name: "duplicate source",
+ annotations: map[string]string{
+ path.Join(MountPrefix, "mount1", "source"): "foo",
+ path.Join(MountPrefix, "mount1", "type"): "tmpfs",
+ path.Join(MountPrefix, "mount1", "share"): "pod",
+
+ path.Join(MountPrefix, "mount2", "source"): "foo",
+ path.Join(MountPrefix, "mount2", "type"): "bind",
+ path.Join(MountPrefix, "mount2", "share"): "container",
+ },
+ error: "have the same mount source",
+ },
+ } {
+ t.Run(tst.name, func(t *testing.T) {
+ spec := &specs.Spec{Annotations: tst.annotations}
+ podHints, err := newPodMountHints(spec)
+ if err == nil || !strings.Contains(err.Error(), tst.error) {
+ t.Errorf("newPodMountHints invalid error, want: .*%s.*, got: %v", tst.error, err)
+ }
+ if podHints != nil {
+ t.Errorf("newPodMountHints must return nil on failure: %+v", podHints)
+ }
+ })
+ }
+}
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index 6ac6b94dd..c1dea736f 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -29,6 +29,7 @@ import (
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
"gvisor.googlesource.com/gvisor/pkg/cpuid"
"gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/memutil"
"gvisor.googlesource.com/gvisor/pkg/rand"
"gvisor.googlesource.com/gvisor/pkg/sentry/arch"
"gvisor.googlesource.com/gvisor/pkg/sentry/control"
@@ -37,7 +38,6 @@ import (
"gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
"gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
"gvisor.googlesource.com/gvisor/pkg/sentry/loader"
- "gvisor.googlesource.com/gvisor/pkg/sentry/memutil"
"gvisor.googlesource.com/gvisor/pkg/sentry/pgalloc"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform/kvm"
@@ -117,6 +117,10 @@ type Loader struct {
//
// processes is guardded by mu.
processes map[execID]*execProcess
+
+ // mountHints provides extra information about mounts for containers that
+ // apply to the entire pod.
+ mountHints *podMountHints
}
// execID uniquely identifies a sentry process that is executed in a container.
@@ -288,7 +292,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 {
@@ -299,18 +303,24 @@ func New(args Args) (*Loader, error) {
return nil, fmt.Errorf("initializing compat logs: %v", err)
}
+ mountHints, err := newPodMountHints(args.Spec)
+ if err != nil {
+ return nil, fmt.Errorf("creating pod mount hints: %v", err)
+ }
+
eid := execID{cid: args.ID}
l := &Loader{
k: k,
conf: args.Conf,
console: args.Console,
- watchdog: watchdog,
+ watchdog: dog,
spec: args.Spec,
goferFDs: args.GoferFDs,
stdioFDs: args.StdioFDs,
rootProcArgs: procArgs,
sandboxID: args.ID,
processes: map[execID]*execProcess{eid: {}},
+ mountHints: mountHints,
}
// We don't care about child signals; some platforms can generate a
@@ -424,6 +434,9 @@ func createMemoryFile() (*pgalloc.MemoryFile, error) {
return nil, fmt.Errorf("error creating memfd: %v", err)
}
memfile := os.NewFile(uintptr(memfd), memfileName)
+ // We can't enable pgalloc.MemoryFileOpts.UseHostMemcgPressure even if
+ // there are memory cgroups specified, because at this point we're already
+ // in a mount namespace in which the relevant cgroupfs is not visible.
mf, err := pgalloc.NewMemoryFile(memfile, pgalloc.MemoryFileOpts{})
if err != nil {
memfile.Close()
@@ -432,7 +445,24 @@ func createMemoryFile() (*pgalloc.MemoryFile, error) {
return mf, nil
}
-// Run runs the root container..
+func (l *Loader) installSeccompFilters() error {
+ if l.conf.DisableSeccomp {
+ filter.Report("syscall filter is DISABLED. Running in less secure mode.")
+ } else {
+ opts := filter.Options{
+ Platform: l.k.Platform,
+ HostNetwork: l.conf.Network == NetworkHost,
+ ProfileEnable: l.conf.ProfileEnable,
+ ControllerFD: l.ctrl.srv.FD(),
+ }
+ if err := filter.Install(opts); err != nil {
+ return fmt.Errorf("installing seccomp filters: %v", err)
+ }
+ }
+ return nil
+}
+
+// Run runs the root container.
func (l *Loader) Run() error {
err := l.run()
l.ctrl.manager.startResultChan <- err
@@ -467,36 +497,34 @@ func (l *Loader) run() error {
return fmt.Errorf("trying to start deleted container %q", l.sandboxID)
}
- // Finally done with all configuration. Setup filters before user code
- // is loaded.
- if l.conf.DisableSeccomp {
- filter.Report("syscall filter is DISABLED. Running in less secure mode.")
- } else {
- opts := filter.Options{
- Platform: l.k.Platform,
- HostNetwork: l.conf.Network == NetworkHost,
- ProfileEnable: l.conf.ProfileEnable,
- ControllerFD: l.ctrl.srv.FD(),
- }
- if err := filter.Install(opts); err != nil {
- return fmt.Errorf("installing seccomp filters: %v", err)
- }
- }
-
// 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 {
+ if l.conf.ProfileEnable {
+ initializePProf()
+ }
+
+ // Finally done with all configuration. Setup filters before user code
+ // is loaded.
+ if err := l.installSeccompFilters(); err != nil {
+ return err
+ }
+
+ // 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, l.mountHints)
+ if err := mntr.setupFS(ctx, l.conf, &l.rootProcArgs, l.rootProcArgs.Credentials); err != nil {
return err
}
@@ -552,7 +580,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 +624,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 +644,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, l.mountHints)
+ 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 +752,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 +764,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
}
diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go
index 4603f751d..2f2499811 100644
--- a/runsc/boot/loader_test.go
+++ b/runsc/boot/loader_test.go
@@ -397,14 +397,15 @@ func TestCreateMountNamespace(t *testing.T) {
}
defer cleanup()
- // setupRootContainerFS needs to find root from the context after the
+ // setupRootContainer needs to find root from the context after the
// namespace is created.
var mns *fs.MountNamespace
setMountNS := func(m *fs.MountNamespace) {
mns = m
ctx.(*contexttest.TestContext).RegisterValue(fs.CtxRoot, mns.Root())
}
- if err := setupRootContainerFS(ctx, ctx, &tc.spec, conf, []int{sandEnd}, setMountNS); err != nil {
+ mntr := newContainerMounter(&tc.spec, "", []int{sandEnd}, nil, &podMountHints{})
+ if err := mntr.setupRootContainer(ctx, ctx, conf, setMountNS); err != nil {
t.Fatalf("createMountNamespace test case %q failed: %v", tc.name, err)
}
root := mns.Root()
@@ -609,8 +610,8 @@ func TestRestoreEnvironment(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
conf := testConfig()
- fds := &fdDispenser{fds: tc.ioFDs}
- actualRenv, err := createRestoreEnvironment(tc.spec, conf, fds)
+ mntr := newContainerMounter(tc.spec, "", tc.ioFDs, nil, &podMountHints{})
+ actualRenv, err := mntr.createRestoreEnvironment(conf)
if !tc.errorExpected && err != nil {
t.Fatalf("could not create restore environment for test:%s", tc.name)
} else if tc.errorExpected {
diff --git a/runsc/boot/network.go b/runsc/boot/network.go
index 0a154d90b..d86803252 100644
--- a/runsc/boot/network.go
+++ b/runsc/boot/network.go
@@ -56,7 +56,11 @@ type FDBasedLink struct {
Addresses []net.IP
Routes []Route
GSOMaxSize uint32
- LinkAddress []byte
+ LinkAddress net.HardwareAddr
+
+ // NumChannels controls how many underlying FD's are to be used to
+ // create this endpoint.
+ NumChannels int
}
// LoopbackLink configures a loopback li nk.
@@ -68,8 +72,9 @@ type LoopbackLink struct {
// CreateLinksAndRoutesArgs are arguments to CreateLinkAndRoutes.
type CreateLinksAndRoutesArgs struct {
- // FilePayload contains the fds associated with the FDBasedLinks. The
- // two slices must have the same length.
+ // FilePayload contains the fds associated with the FDBasedLinks. The
+ // number of fd's should match the sum of the NumChannels field of the
+ // FDBasedLink entries below.
urpc.FilePayload
LoopbackLinks []LoopbackLink
@@ -95,8 +100,12 @@ func (r *Route) toTcpipRoute(id tcpip.NICID) tcpip.Route {
// CreateLinksAndRoutes creates links and routes in a network stack. It should
// only be called once.
func (n *Network) CreateLinksAndRoutes(args *CreateLinksAndRoutesArgs, _ *struct{}) error {
- if len(args.FilePayload.Files) != len(args.FDBasedLinks) {
- return fmt.Errorf("FilePayload must be same length at FDBasedLinks")
+ wantFDs := 0
+ for _, l := range args.FDBasedLinks {
+ wantFDs += l.NumChannels
+ }
+ if got := len(args.FilePayload.Files); got != wantFDs {
+ return fmt.Errorf("args.FilePayload.Files has %d FD's but we need %d entries based on FDBasedLinks", got, wantFDs)
}
var nicID tcpip.NICID
@@ -123,20 +132,26 @@ func (n *Network) CreateLinksAndRoutes(args *CreateLinksAndRoutesArgs, _ *struct
}
}
- for i, link := range args.FDBasedLinks {
+ fdOffset := 0
+ for _, link := range args.FDBasedLinks {
nicID++
nicids[link.Name] = nicID
- // Copy the underlying FD.
- oldFD := args.FilePayload.Files[i].Fd()
- newFD, err := syscall.Dup(int(oldFD))
- if err != nil {
- return fmt.Errorf("failed to dup FD %v: %v", oldFD, err)
+ FDs := []int{}
+ for j := 0; j < link.NumChannels; j++ {
+ // Copy the underlying FD.
+ oldFD := args.FilePayload.Files[fdOffset].Fd()
+ newFD, err := syscall.Dup(int(oldFD))
+ if err != nil {
+ return fmt.Errorf("failed to dup FD %v: %v", oldFD, err)
+ }
+ FDs = append(FDs, newFD)
+ fdOffset++
}
mac := tcpip.LinkAddress(link.LinkAddress)
linkEP, err := fdbased.New(&fdbased.Options{
- FD: newFD,
+ FDs: FDs,
MTU: uint32(link.MTU),
EthernetHeader: true,
Address: mac,
@@ -148,7 +163,7 @@ func (n *Network) CreateLinksAndRoutes(args *CreateLinksAndRoutesArgs, _ *struct
return err
}
- log.Infof("Enabling interface %q with id %d on addresses %+v (%v)", link.Name, nicID, link.Addresses, mac)
+ log.Infof("Enabling interface %q with id %d on addresses %+v (%v) w/ %d channels", link.Name, nicID, link.Addresses, mac, link.NumChannels)
if err := n.createNICWithAddrs(nicID, link.Name, linkEP, link.Addresses, false /* loopback */); err != nil {
return err
}
diff --git a/runsc/boot/pprof.go b/runsc/boot/pprof.go
new file mode 100644
index 000000000..463362f02
--- /dev/null
+++ b/runsc/boot/pprof.go
@@ -0,0 +1,18 @@
+// Copyright 2019 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package boot
+
+func initializePProf() {
+}