summaryrefslogtreecommitdiffhomepage
path: root/runsc/boot
diff options
context:
space:
mode:
Diffstat (limited to 'runsc/boot')
-rw-r--r--runsc/boot/BUILD2
-rw-r--r--runsc/boot/controller.go17
-rw-r--r--runsc/boot/filter/config.go11
-rw-r--r--runsc/boot/fs.go18
-rw-r--r--runsc/boot/loader.go146
-rw-r--r--runsc/boot/loader_test.go2
-rw-r--r--runsc/boot/vfs.go25
7 files changed, 119 insertions, 102 deletions
diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD
index d51347fe1..c7b26746b 100644
--- a/runsc/boot/BUILD
+++ b/runsc/boot/BUILD
@@ -32,13 +32,13 @@ go_library(
"//pkg/control/server",
"//pkg/coverage",
"//pkg/cpuid",
+ "//pkg/errors/linuxerr",
"//pkg/eventchannel",
"//pkg/fd",
"//pkg/flipcall",
"//pkg/fspath",
"//pkg/log",
"//pkg/memutil",
- "//pkg/metric",
"//pkg/rand",
"//pkg/refs",
"//pkg/refsvfs2",
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go
index 9b270cbf2..132973e6b 100644
--- a/runsc/boot/controller.go
+++ b/runsc/boot/controller.go
@@ -18,6 +18,7 @@ import (
"errors"
"fmt"
"os"
+ gtime "time"
specs "github.com/opencontainers/runtime-spec/specs-go"
"golang.org/x/sys/unix"
@@ -165,8 +166,11 @@ func newController(fd int, l *Loader) (*controller, error) {
return ctrl, nil
}
+// stopRPCTimeout is the time for clients to complete ongoing RPCs.
+const stopRPCTimeout = 15 * gtime.Second
+
func (c *controller) stop() {
- c.srv.Stop()
+ c.srv.Stop(stopRPCTimeout)
}
// containerManager manages sandbox containers.
@@ -330,6 +334,11 @@ func (cm *containerManager) ExecuteAsync(args *control.ExecArgs, pid *int32) err
// Checkpoint pauses a sandbox and saves its state.
func (cm *containerManager) Checkpoint(o *control.SaveOpts, _ *struct{}) error {
log.Debugf("containerManager.Checkpoint")
+ // TODO(gvisor.dev/issues/6243): save/restore not supported w/ hostinet
+ if cm.l.root.conf.Network == config.NetworkHost {
+ return errors.New("checkpoint not supported when using hostinet")
+ }
+
state := control.State{
Kernel: cm.l.k,
Watchdog: cm.l.watchdog,
@@ -340,6 +349,10 @@ func (cm *containerManager) Checkpoint(o *control.SaveOpts, _ *struct{}) error {
// Pause suspends a container.
func (cm *containerManager) Pause(_, _ *struct{}) error {
log.Debugf("containerManager.Pause")
+ // TODO(gvisor.dev/issues/6243): save/restore not supported w/ hostinet
+ if cm.l.root.conf.Network == config.NetworkHost {
+ return errors.New("pause not supported when using hostinet")
+ }
cm.l.k.Pause()
return nil
}
@@ -439,7 +452,7 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error {
// Load the state.
loadOpts := state.LoadOpts{Source: specFile}
- if err := loadOpts.Load(ctx, k, networkStack, time.NewCalibratedClocks(), &vfs.CompleteRestoreOptions{}); err != nil {
+ if err := loadOpts.Load(ctx, k, nil, networkStack, time.NewCalibratedClocks(), &vfs.CompleteRestoreOptions{}); err != nil {
return err
}
diff --git a/runsc/boot/filter/config.go b/runsc/boot/filter/config.go
index 49b503f99..33e738efc 100644
--- a/runsc/boot/filter/config.go
+++ b/runsc/boot/filter/config.go
@@ -200,6 +200,12 @@ var allowedSyscalls = seccomp.SyscallRules{
seccomp.MatchAny{},
seccomp.MatchAny{},
seccomp.MatchAny{},
+ seccomp.EqualTo(unix.MAP_SHARED | unix.MAP_FIXED),
+ },
+ {
+ seccomp.MatchAny{},
+ seccomp.MatchAny{},
+ seccomp.MatchAny{},
seccomp.EqualTo(unix.MAP_PRIVATE),
},
{
@@ -265,7 +271,6 @@ var allowedSyscalls = seccomp.SyscallRules{
seccomp.MatchAny{},
seccomp.MatchAny{},
seccomp.EqualTo(unix.MSG_DONTWAIT),
- seccomp.EqualTo(0),
},
},
unix.SYS_RESTART_SYSCALL: {},
@@ -454,6 +459,10 @@ func hostInetFilters() seccomp.SyscallRules {
seccomp.MatchAny{},
seccomp.EqualTo(unix.TIOCINQ),
},
+ {
+ seccomp.MatchAny{},
+ seccomp.EqualTo(unix.SIOCGIFFLAGS),
+ },
},
unix.SYS_LISTEN: {},
unix.SYS_READV: {},
diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go
index bf4a41f77..7fce2b708 100644
--- a/runsc/boot/fs.go
+++ b/runsc/boot/fs.go
@@ -25,6 +25,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -41,7 +42,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/runsc/config"
"gvisor.dev/gvisor/runsc/specutils"
@@ -763,12 +763,10 @@ func (c *containerMounter) createRootMount(ctx context.Context, conf *config.Con
p9FS := mustFindFilesystem("9p")
opts := p9MountData(fd, conf.FileAccess, false /* vfs2 */)
- if conf.OverlayfsStaleRead {
- // We can't check for overlayfs here because sandbox is chroot'ed and gofer
- // can only send mount options for specs.Mounts (specs.Root is missing
- // Options field). So assume root is always on top of overlayfs.
- opts = append(opts, "overlayfs_stale_read")
- }
+ // We can't check for overlayfs here because sandbox is chroot'ed and gofer
+ // can only send mount options for specs.Mounts (specs.Root is missing
+ // Options field). So assume root is always on top of overlayfs.
+ opts = append(opts, "overlayfs_stale_read")
rootInode, err := p9FS.Mount(ctx, rootDevice, mf, strings.Join(opts, ","), nil)
if err != nil {
@@ -1041,8 +1039,8 @@ func (c *containerMounter) mountTmp(ctx context.Context, conf *config.Config, mn
maxTraversals := uint(0)
tmp, err := mns.FindInode(ctx, root, root, "tmp", &maxTraversals)
- switch err {
- case nil:
+ switch {
+ case err == nil:
// Found '/tmp' in filesystem, check if it's empty.
defer tmp.DecRef(ctx)
f, err := tmp.Inode.GetFile(ctx, tmp, fs.FileFlags{Read: true, Directory: true})
@@ -1063,7 +1061,7 @@ func (c *containerMounter) mountTmp(ctx context.Context, conf *config.Config, mn
log.Infof("Mounting internal tmpfs on top of empty %q", "/tmp")
fallthrough
- case syserror.ENOENT:
+ case linuxerr.Equals(linuxerr.ENOENT, err):
// No '/tmp' found (or fallthrough from above). Safe to mount internal
// tmpfs.
tmpMount := specs.Mount{
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index 10f2d3d35..8d71d7447 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -34,11 +34,9 @@ import (
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/memutil"
- "gvisor.dev/gvisor/pkg/metric"
"gvisor.dev/gvisor/pkg/rand"
"gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/refsvfs2"
- "gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/control"
"gvisor.dev/gvisor/pkg/sentry/fdimport"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -211,15 +209,13 @@ func New(args Args) (*Loader, error) {
// We initialize the rand package now to make sure /dev/urandom is pre-opened
// on kernels that do not support getrandom(2).
if err := rand.Init(); err != nil {
- return nil, fmt.Errorf("setting up rand: %v", err)
+ return nil, fmt.Errorf("setting up rand: %w", err)
}
if err := usage.Init(); err != nil {
- return nil, fmt.Errorf("setting up memory usage: %v", err)
+ return nil, fmt.Errorf("setting up memory usage: %w", err)
}
- metric.CreateSentryMetrics()
-
// Is this a VFSv2 kernel?
if args.Conf.VFS2 {
kernel.VFS2Enabled = true
@@ -260,7 +256,7 @@ func New(args Args) (*Loader, error) {
// Create kernel and platform.
p, err := createPlatform(args.Conf, args.Device)
if err != nil {
- return nil, fmt.Errorf("creating platform: %v", err)
+ return nil, fmt.Errorf("creating platform: %w", err)
}
k := &kernel.Kernel{
Platform: p,
@@ -269,7 +265,7 @@ func New(args Args) (*Loader, error) {
// Create memory file.
mf, err := createMemoryFile()
if err != nil {
- return nil, fmt.Errorf("creating memory file: %v", err)
+ return nil, fmt.Errorf("creating memory file: %w", err)
}
k.SetMemoryFile(mf)
@@ -278,30 +274,27 @@ func New(args Args) (*Loader, error) {
// Pass k as the platform since it is savable, unlike the actual platform.
vdso, err := loader.PrepareVDSO(k)
if err != nil {
- return nil, fmt.Errorf("creating vdso: %v", err)
+ return nil, fmt.Errorf("creating vdso: %w", err)
}
// Create timekeeper.
- tk, err := kernel.NewTimekeeper(k, vdso.ParamPage.FileRange())
- if err != nil {
- return nil, fmt.Errorf("creating timekeeper: %v", err)
- }
+ tk := kernel.NewTimekeeper(k, vdso.ParamPage.FileRange())
tk.SetClocks(time.NewCalibratedClocks())
if err := enableStrace(args.Conf); err != nil {
- return nil, fmt.Errorf("enabling strace: %v", err)
+ return nil, fmt.Errorf("enabling strace: %w", err)
}
// Create root network namespace/stack.
- netns, err := newRootNetworkNamespace(args.Conf, k, k)
+ netns, err := newRootNetworkNamespace(args.Conf, tk, k)
if err != nil {
- return nil, fmt.Errorf("creating network: %v", err)
+ return nil, fmt.Errorf("creating network: %w", err)
}
// Create capabilities.
caps, err := specutils.Capabilities(args.Conf.EnableRaw, args.Spec.Process.Capabilities)
if err != nil {
- return nil, fmt.Errorf("converting capabilities: %v", err)
+ return nil, fmt.Errorf("converting capabilities: %w", err)
}
// Convert the spec's additional GIDs to KGIDs.
@@ -345,7 +338,7 @@ func New(args Args) (*Loader, error) {
RootAbstractSocketNamespace: kernel.NewAbstractSocketNamespace(),
PIDNamespace: kernel.NewRootPIDNamespace(creds.UserNamespace),
}); err != nil {
- return nil, fmt.Errorf("initializing kernel: %v", err)
+ return nil, fmt.Errorf("initializing kernel: %w", err)
}
if kernel.VFS2Enabled {
@@ -374,17 +367,17 @@ func New(args Args) (*Loader, error) {
procArgs, err := createProcessArgs(args.ID, args.Spec, creds, k, k.RootPIDNamespace())
if err != nil {
- return nil, fmt.Errorf("creating init process for root container: %v", err)
+ return nil, fmt.Errorf("creating init process for root container: %w", err)
}
info.procArgs = procArgs
if err := initCompatLogs(args.UserLogFD); err != nil {
- return nil, fmt.Errorf("initializing compat logs: %v", err)
+ return nil, fmt.Errorf("initializing compat logs: %w", err)
}
mountHints, err := newPodMountHints(args.Spec)
if err != nil {
- return nil, fmt.Errorf("creating pod mount hints: %v", err)
+ return nil, fmt.Errorf("creating pod mount hints: %w", err)
}
info.conf = args.Conf
@@ -394,12 +387,12 @@ func New(args Args) (*Loader, error) {
// Set up host mount that will be used for imported fds.
hostFilesystem, err := hostvfs2.NewFilesystem(k.VFS())
if err != nil {
- return nil, fmt.Errorf("failed to create hostfs filesystem: %v", err)
+ return nil, fmt.Errorf("failed to create hostfs filesystem: %w", err)
}
defer hostFilesystem.DecRef(k.SupervisorContext())
hostMount, err := k.VFS().NewDisconnectedMount(hostFilesystem, nil, &vfs.MountOptions{})
if err != nil {
- return nil, fmt.Errorf("failed to create hostfs mount: %v", err)
+ return nil, fmt.Errorf("failed to create hostfs mount: %w", err)
}
k.SetHostMount(hostMount)
}
@@ -417,7 +410,7 @@ func New(args Args) (*Loader, error) {
// We don't care about child signals; some platforms can generate a
// tremendous number of useless ones (I'm looking at you, ptrace).
if err := sighandling.IgnoreChildStop(); err != nil {
- return nil, fmt.Errorf("ignore child stop signals failed: %v", err)
+ return nil, fmt.Errorf("ignore child stop signals failed: %w", err)
}
// Create the control server using the provided FD.
@@ -426,14 +419,14 @@ func New(args Args) (*Loader, error) {
// controller is used to configure the kernel's network stack.
ctrl, err := newController(args.ControllerFD, l)
if err != nil {
- return nil, fmt.Errorf("creating control server: %v", err)
+ return nil, fmt.Errorf("creating control server: %w", err)
}
l.ctrl = ctrl
// Only start serving after Loader is set to controller and controller is set
// to Loader, because they are both used in the urpc methods.
if err := ctrl.srv.StartServing(); err != nil {
- return nil, fmt.Errorf("starting control server: %v", err)
+ return nil, fmt.Errorf("starting control server: %w", err)
}
return l, nil
@@ -444,7 +437,7 @@ func createProcessArgs(id string, spec *specs.Spec, creds *auth.Credentials, k *
// Create initial limits.
ls, err := createLimitSet(spec)
if err != nil {
- return kernel.CreateProcessArgs{}, fmt.Errorf("creating limits: %v", err)
+ return kernel.CreateProcessArgs{}, fmt.Errorf("creating limits: %w", err)
}
env, err := specutils.ResolveEnvs(spec.Process.Env)
if err != nil {
@@ -498,18 +491,18 @@ func (l *Loader) Destroy() {
// In the success case, stdioFDs and goferFDs will only contain
// released/closed FDs that ownership has been passed over to host FDs and
// gofer sessions. Close them here in case of failure.
- for _, fd := range l.root.stdioFDs {
- _ = fd.Close()
+ for _, f := range l.root.stdioFDs {
+ _ = f.Close()
}
- for _, fd := range l.root.goferFDs {
- _ = fd.Close()
+ for _, f := range l.root.goferFDs {
+ _ = f.Close()
}
}
func createPlatform(conf *config.Config, deviceFile *os.File) (platform.Platform, error) {
p, err := platform.Lookup(conf.Platform)
if err != nil {
- panic(fmt.Sprintf("invalid platform %v: %v", conf.Platform, err))
+ panic(fmt.Sprintf("invalid platform %s: %s", conf.Platform, err))
}
log.Infof("Platform: %s", conf.Platform)
return p.New(deviceFile)
@@ -519,7 +512,7 @@ func createMemoryFile() (*pgalloc.MemoryFile, error) {
const memfileName = "runsc-memory"
memfd, err := memutil.CreateMemFD(memfileName, 0)
if err != nil {
- return nil, fmt.Errorf("error creating memfd: %v", err)
+ return nil, fmt.Errorf("error creating memfd: %w", err)
}
memfile := os.NewFile(uintptr(memfd), memfileName)
// We can't enable pgalloc.MemoryFileOpts.UseHostMemcgPressure even if
@@ -527,8 +520,8 @@ func createMemoryFile() (*pgalloc.MemoryFile, error) {
// in a mount namespace in which the relevant cgroupfs is not visible.
mf, err := pgalloc.NewMemoryFile(memfile, pgalloc.MemoryFileOpts{})
if err != nil {
- memfile.Close()
- return nil, fmt.Errorf("error creating pgalloc.MemoryFile: %v", err)
+ _ = memfile.Close()
+ return nil, fmt.Errorf("error creating pgalloc.MemoryFile: %w", err)
}
return mf, nil
}
@@ -545,7 +538,7 @@ func (l *Loader) installSeccompFilters() error {
ControllerFD: l.ctrl.srv.FD(),
}
if err := filter.Install(opts); err != nil {
- return fmt.Errorf("installing seccomp filters: %v", err)
+ return fmt.Errorf("installing seccomp filters: %w", err)
}
}
return nil
@@ -571,8 +564,8 @@ func (l *Loader) run() error {
// Delay host network configuration to this point because network namespace
// is configured after the loader is created and before Run() is called.
log.Debugf("Configuring host network")
- stack := l.k.RootNetworkNamespace().Stack().(*hostinet.Stack)
- if err := stack.Configure(); err != nil {
+ s := l.k.RootNetworkNamespace().Stack().(*hostinet.Stack)
+ if err := s.Configure(); err != nil {
return err
}
}
@@ -629,9 +622,9 @@ func (l *Loader) run() error {
// be handled properly.
deliveryMode = DeliverToForegroundProcessGroup
}
- log.Infof("Received external signal %d, mode: %v", sig, deliveryMode)
+ log.Infof("Received external signal %d, mode: %s", sig, deliveryMode)
if err := l.signal(l.sandboxID, 0, int32(sig), deliveryMode); err != nil {
- log.Warningf("error sending signal %v to container %q: %v", sig, l.sandboxID, err)
+ log.Warningf("error sending signal %s to container %q: %s", sig, l.sandboxID, err)
}
})
@@ -660,7 +653,7 @@ func (l *Loader) startContainer(spec *specs.Spec, conf *config.Config, cid strin
// Create capabilities.
caps, err := specutils.Capabilities(conf.EnableRaw, spec.Process.Capabilities)
if err != nil {
- return fmt.Errorf("creating capabilities: %v", err)
+ return fmt.Errorf("creating capabilities: %w", err)
}
l.mu.Lock()
@@ -713,16 +706,16 @@ func (l *Loader) startContainer(spec *specs.Spec, conf *config.Config, cid strin
}
info.procArgs, err = createProcessArgs(cid, spec, creds, l.k, pidns)
if err != nil {
- return fmt.Errorf("creating new process: %v", err)
+ return fmt.Errorf("creating new process: %w", err)
}
// Use stdios or TTY depending on the spec configuration.
if spec.Process.Terminal {
- if len(stdioFDs) > 0 {
- return fmt.Errorf("using TTY, stdios not expected: %v", stdioFDs)
+ if l := len(stdioFDs); l != 0 {
+ return fmt.Errorf("using TTY, stdios not expected: %d", l)
}
if ep.hostTTY == nil {
- return fmt.Errorf("terminal enabled but no TTY provided. Did you set --console-socket on create?")
+ return fmt.Errorf("terminal enabled but no TTY provided (--console-socket possibly passed)")
}
info.stdioFDs = []*fd.FD{ep.hostTTY, ep.hostTTY, ep.hostTTY}
ep.hostTTY = nil
@@ -743,7 +736,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn
ctx := info.procArgs.NewContext(l.k)
fdTable, ttyFile, ttyFileVFS2, err := createFDTable(ctx, info.spec.Process.Terminal, info.stdioFDs)
if err != nil {
- return nil, nil, nil, fmt.Errorf("importing fds: %v", err)
+ return nil, nil, nil, fmt.Errorf("importing fds: %w", err)
}
// CreateProcess takes a reference on fdTable if successful. We won't need
// ours either way.
@@ -780,7 +773,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn
// Create and start the new process.
tg, _, err := l.k.CreateProcess(info.procArgs)
if err != nil {
- return nil, nil, nil, fmt.Errorf("creating process: %v", err)
+ return nil, nil, nil, fmt.Errorf("creating process: %w", err)
}
// CreateProcess takes a reference on FDTable if successful.
info.procArgs.FDTable.DecRef(ctx)
@@ -799,7 +792,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn
if info.spec.Linux != nil && info.spec.Linux.Seccomp != nil {
program, err := seccomp.BuildProgram(info.spec.Linux.Seccomp)
if err != nil {
- return nil, nil, nil, fmt.Errorf("building seccomp program: %v", err)
+ return nil, nil, nil, fmt.Errorf("building seccomp program: %w", err)
}
if log.IsLogging(log.Debug) {
@@ -810,7 +803,7 @@ func (l *Loader) createContainerProcess(root bool, cid string, info *containerIn
task := tg.Leader()
// NOTE: It seems Flags are ignored by runc so we ignore them too.
if err := task.AppendSyscallFilter(program, true); err != nil {
- return nil, nil, nil, fmt.Errorf("appending seccomp filters: %v", err)
+ return nil, nil, nil, fmt.Errorf("appending seccomp filters: %w", err)
}
}
} else {
@@ -841,7 +834,7 @@ func (l *Loader) startGoferMonitor(cid string, goferFDs []*fd.FD) {
return uintptr(n), 0, err
})
if err != nil {
- panic(fmt.Sprintf("Error monitoring gofer FDs: %v", err))
+ panic(fmt.Sprintf("Error monitoring gofer FDs: %s", err))
}
l.mu.Lock()
@@ -852,7 +845,7 @@ func (l *Loader) startGoferMonitor(cid string, goferFDs []*fd.FD) {
if tg, _ := l.tryThreadGroupFromIDLocked(execID{cid: cid}); tg != nil {
log.Infof("Gofer socket disconnected, killing container %q", cid)
if err := l.signalAllProcesses(cid, int32(linux.SIGKILL)); err != nil {
- log.Warningf("Error killing container %q after gofer stopped: %v", cid, err)
+ log.Warningf("Error killing container %q after gofer stopped: %s", cid, err)
}
}
}()
@@ -873,7 +866,7 @@ func (l *Loader) destroyContainer(cid string) error {
// The container exists, but has it been started?
if tg != nil {
if err := l.signalAllProcesses(cid, int32(linux.SIGKILL)); err != nil {
- return fmt.Errorf("sending SIGKILL to all container processes: %v", err)
+ return fmt.Errorf("sending SIGKILL to all container processes: %w", err)
}
// Wait for all processes that belong to the container to exit (including
// exec'd processes).
@@ -967,10 +960,15 @@ func (l *Loader) executeAsync(args *control.ExecArgs) (kernel.ThreadID, error) {
}
args.Envv = envv
}
+ args.PIDNamespace = tg.PIDNamespace()
+
+ args.Limits, err = createLimitSet(l.root.spec)
+ if err != nil {
+ return 0, fmt.Errorf("creating limits: %w", err)
+ }
// Start the process.
proc := control.Proc{Kernel: l.k}
- args.PIDNamespace = tg.PIDNamespace()
newTG, tgid, ttyFile, ttyFileVFS2, err := control.ExecAsync(&proc, args)
if err != nil {
return 0, err
@@ -982,7 +980,7 @@ func (l *Loader) executeAsync(args *control.ExecArgs) (kernel.ThreadID, error) {
tty: ttyFile,
ttyVFS2: ttyFileVFS2,
}
- log.Debugf("updated processes: %v", l.processes)
+ log.Debugf("updated processes: %s", l.processes)
return tgid, nil
}
@@ -993,7 +991,7 @@ func (l *Loader) waitContainer(cid string, waitStatus *uint32) error {
// multiple clients to wait on the same container.
tg, err := l.threadGroupFromID(execID{cid: cid})
if err != nil {
- return fmt.Errorf("can't wait for container %q: %v", cid, err)
+ return fmt.Errorf("can't wait for container %q: %w", cid, err)
}
// If the thread either has already exited or exits during waiting,
@@ -1007,7 +1005,7 @@ func (l *Loader) waitContainer(cid string, waitStatus *uint32) error {
if l.root.procArgs.ContainerID == cid {
// All sentry-created resources should have been released at this point.
refsvfs2.DoLeakCheck()
- coverage.Report()
+ _ = coverage.Report()
}
return nil
}
@@ -1026,7 +1024,7 @@ func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, waitStatus *uint32) e
l.mu.Lock()
delete(l.processes, eid)
- log.Debugf("updated processes (removal): %v", l.processes)
+ log.Debugf("updated processes (removal): %s", l.processes)
l.mu.Unlock()
return nil
}
@@ -1035,7 +1033,7 @@ func (l *Loader) waitPID(tgid kernel.ThreadID, cid string, waitStatus *uint32) e
// In this case, find the process in the container's PID namespace.
initTG, err := l.threadGroupFromID(execID{cid: cid})
if err != nil {
- return fmt.Errorf("waiting for PID %d: %v", tgid, err)
+ return fmt.Errorf("waiting for PID %d: %w", tgid, err)
}
tg := initTG.PIDNamespace().ThreadGroupWithID(tgid)
if tg == nil {
@@ -1094,7 +1092,7 @@ func newRootNetworkNamespace(conf *config.Config, clock tcpip.Clock, uniqueID st
return inet.NewRootNamespace(s, creator), nil
default:
- panic(fmt.Sprintf("invalid network configuration: %v", conf.Network))
+ panic(fmt.Sprintf("invalid network configuration: %d", conf.Network))
}
}
@@ -1107,7 +1105,7 @@ func newEmptySandboxNetworkStack(clock tcpip.Clock, uniqueID stack.UniqueID) (in
icmp.NewProtocol4,
icmp.NewProtocol6,
}
- s := netstack.Stack{stack.New(stack.Options{
+ s := netstack.Stack{Stack: stack.New(stack.Options{
NetworkProtocols: netProtos,
TransportProtocols: transProtos,
Clock: clock,
@@ -1115,9 +1113,9 @@ func newEmptySandboxNetworkStack(clock tcpip.Clock, uniqueID stack.UniqueID) (in
HandleLocal: true,
// Enable raw sockets for users with sufficient
// privileges.
- RawFactory: raw.EndpointFactory{},
- UniqueID: uniqueID,
- IPTables: netfilter.DefaultLinuxTables(),
+ RawFactory: raw.EndpointFactory{},
+ UniqueID: uniqueID,
+ DefaultIPTables: netfilter.DefaultLinuxTables,
})}
// Enable SACK Recovery.
@@ -1190,13 +1188,13 @@ func (l *Loader) signal(cid string, pid, signo int32, mode SignalDeliveryMode) e
switch mode {
case DeliverToProcess:
if err := l.signalProcess(cid, kernel.ThreadID(pid), signo); err != nil {
- return fmt.Errorf("signaling process in container %q PID %d: %v", cid, pid, err)
+ return fmt.Errorf("signaling process in container %q PID %d: %w", cid, pid, err)
}
return nil
case DeliverToForegroundProcessGroup:
if err := l.signalForegrondProcessGroup(cid, kernel.ThreadID(pid), signo); err != nil {
- return fmt.Errorf("signaling foreground process group in container %q PID %d: %v", cid, pid, err)
+ return fmt.Errorf("signaling foreground process group in container %q PID %d: %w", cid, pid, err)
}
return nil
@@ -1209,12 +1207,12 @@ func (l *Loader) signal(cid string, pid, signo int32, mode SignalDeliveryMode) e
return err
}
if err := l.signalAllProcesses(cid, signo); err != nil {
- return fmt.Errorf("signaling all processes in container %q: %v", cid, err)
+ return fmt.Errorf("signaling all processes in container %q: %w", cid, err)
}
return nil
default:
- panic(fmt.Sprintf("unknown signal delivery mode %v", mode))
+ panic(fmt.Sprintf("unknown signal delivery mode %s", mode))
}
}
@@ -1224,7 +1222,7 @@ func (l *Loader) signalProcess(cid string, tgid kernel.ThreadID, signo int32) er
execTG, err := l.threadGroupFromID(execID{cid: cid, pid: tgid})
if err == nil {
// Send signal directly to the identified process.
- return l.k.SendExternalSignalThreadGroup(execTG, &arch.SignalInfo{Signo: signo})
+ return l.k.SendExternalSignalThreadGroup(execTG, &linux.SignalInfo{Signo: signo})
}
// The caller may be signaling a process not started directly via exec.
@@ -1237,7 +1235,7 @@ func (l *Loader) signalProcess(cid string, tgid kernel.ThreadID, signo int32) er
if tg.Leader().ContainerID() != cid {
return fmt.Errorf("process %d belongs to a different container: %q", tgid, tg.Leader().ContainerID())
}
- return l.k.SendExternalSignalThreadGroup(tg, &arch.SignalInfo{Signo: signo})
+ return l.k.SendExternalSignalThreadGroup(tg, &linux.SignalInfo{Signo: signo})
}
// signalForegrondProcessGroup looks up foreground process group from the TTY
@@ -1247,7 +1245,7 @@ func (l *Loader) signalForegrondProcessGroup(cid string, tgid kernel.ThreadID, s
tg, err := l.tryThreadGroupFromIDLocked(execID{cid: cid, pid: tgid})
if err != nil {
l.mu.Unlock()
- return fmt.Errorf("no thread group found: %v", err)
+ return fmt.Errorf("no thread group found: %w", err)
}
if tg == nil {
l.mu.Unlock()
@@ -1257,7 +1255,7 @@ func (l *Loader) signalForegrondProcessGroup(cid string, tgid kernel.ThreadID, s
tty, ttyVFS2, err := l.ttyFromIDLocked(execID{cid: cid, pid: tgid})
l.mu.Unlock()
if err != nil {
- return fmt.Errorf("no thread group found: %v", err)
+ return fmt.Errorf("no thread group found: %w", err)
}
var pg *kernel.ProcessGroup
@@ -1273,7 +1271,7 @@ func (l *Loader) signalForegrondProcessGroup(cid string, tgid kernel.ThreadID, s
// No foreground process group has been set. Signal the
// original thread group.
log.Warningf("No foreground process group for container %q and PID %d. Sending signal directly to PID %d.", cid, tgid, tgid)
- return l.k.SendExternalSignalThreadGroup(tg, &arch.SignalInfo{Signo: signo})
+ return l.k.SendExternalSignalThreadGroup(tg, &linux.SignalInfo{Signo: signo})
}
// Send the signal to all processes in the process group.
var lastErr error
@@ -1281,7 +1279,7 @@ func (l *Loader) signalForegrondProcessGroup(cid string, tgid kernel.ThreadID, s
if tg.ProcessGroup() != pg {
continue
}
- if err := l.k.SendExternalSignalThreadGroup(tg, &arch.SignalInfo{Signo: signo}); err != nil {
+ if err := l.k.SendExternalSignalThreadGroup(tg, &linux.SignalInfo{Signo: signo}); err != nil {
lastErr = err
}
}
@@ -1296,7 +1294,7 @@ func (l *Loader) signalAllProcesses(cid string, signo int32) error {
// sent to the entire container.
l.k.Pause()
defer l.k.Unpause()
- return l.k.SendContainerSignal(cid, &arch.SignalInfo{Signo: signo})
+ return l.k.SendContainerSignal(cid, &linux.SignalInfo{Signo: signo})
}
// threadGroupFromID is similar to tryThreadGroupFromIDLocked except that it
diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go
index 93c476971..b5e8d08a5 100644
--- a/runsc/boot/loader_test.go
+++ b/runsc/boot/loader_test.go
@@ -214,7 +214,7 @@ func doStartSignal(t *testing.T, vfsEnabled bool) {
// We aren't going to wait on this application, so the control server
// needs to be shut down manually.
- defer l.ctrl.srv.Stop()
+ defer l.ctrl.srv.Stop(time.Hour)
// Start a goroutine that calls WaitForStartSignal and writes to a
// channel when it returns.
diff --git a/runsc/boot/vfs.go b/runsc/boot/vfs.go
index 7be5176b0..ca1a86e39 100644
--- a/runsc/boot/vfs.go
+++ b/runsc/boot/vfs.go
@@ -24,6 +24,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/cleanup"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/devices/memdev"
@@ -210,12 +211,10 @@ func (c *containerMounter) createMountNamespaceVFS2(ctx context.Context, conf *c
fd := c.fds.remove()
data := p9MountData(fd, conf.FileAccess, true /* vfs2 */)
- if conf.OverlayfsStaleRead {
- // We can't check for overlayfs here because sandbox is chroot'ed and gofer
- // can only send mount options for specs.Mounts (specs.Root is missing
- // Options field). So assume root is always on top of overlayfs.
- data = append(data, "overlayfs_stale_read")
- }
+ // We can't check for overlayfs here because sandbox is chroot'ed and gofer
+ // can only send mount options for specs.Mounts (specs.Root is missing
+ // Options field). So assume root is always on top of overlayfs.
+ data = append(data, "overlayfs_stale_read")
log.Infof("Mounting root over 9P, ioFD: %d", fd)
opts := &vfs.MountOptions{
@@ -658,8 +657,8 @@ func (c *containerMounter) mountTmpVFS2(ctx context.Context, conf *config.Config
Path: fspath.Parse("/tmp"),
}
fd, err := c.k.VFS().OpenAt(ctx, creds, &pop, &vfs.OpenOptions{Flags: linux.O_RDONLY | linux.O_DIRECTORY})
- switch err {
- case nil:
+ switch {
+ case err == nil:
defer fd.DecRef(ctx)
err := fd.IterDirents(ctx, vfs.IterDirentsCallbackFunc(func(dirent vfs.Dirent) error {
@@ -668,10 +667,10 @@ func (c *containerMounter) mountTmpVFS2(ctx context.Context, conf *config.Config
}
return nil
}))
- switch err {
- case nil:
+ switch {
+ case err == nil:
log.Infof(`Mounting internal tmpfs on top of empty "/tmp"`)
- case syserror.ENOTEMPTY:
+ case linuxerr.Equals(linuxerr.ENOTEMPTY, err):
// If more than "." and ".." is found, skip internal tmpfs to prevent
// hiding existing files.
log.Infof(`Skipping internal tmpfs mount for "/tmp" because it's not empty`)
@@ -681,7 +680,7 @@ func (c *containerMounter) mountTmpVFS2(ctx context.Context, conf *config.Config
}
fallthrough
- case syserror.ENOENT:
+ case linuxerr.Equals(linuxerr.ENOENT, err):
// No '/tmp' found (or fallthrough from above). It's safe to mount internal
// tmpfs.
tmpMount := specs.Mount{
@@ -694,7 +693,7 @@ func (c *containerMounter) mountTmpVFS2(ctx context.Context, conf *config.Config
_, err := c.mountSubmountVFS2(ctx, conf, mns, creds, &mountAndFD{mount: &tmpMount})
return err
- case syserror.ENOTDIR:
+ case linuxerr.Equals(linuxerr.ENOTDIR, err):
// Not a dir?! Let it be.
return nil