summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/kernel/kernel.go
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/kernel/kernel.go')
-rw-r--r--pkg/sentry/kernel/kernel.go513
1 files changed, 407 insertions, 106 deletions
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index fcfe7a16d..1028d13c6 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -34,21 +34,25 @@ package kernel
import (
"errors"
"fmt"
- "io"
"path/filepath"
- "sync"
"sync/atomic"
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/cpuid"
"gvisor.dev/gvisor/pkg/eventchannel"
+ "gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/sentry/arch"
- "gvisor.dev/gvisor/pkg/sentry/context"
"gvisor.dev/gvisor/pkg/sentry/fs"
- "gvisor.dev/gvisor/pkg/sentry/fs/timerfd"
+ oldtimerfd "gvisor.dev/gvisor/pkg/sentry/fs/timerfd"
+ "gvisor.dev/gvisor/pkg/sentry/fsbridge"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/pipefs"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/sockfs"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/timerfd"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/tmpfs"
"gvisor.dev/gvisor/pkg/sentry/hostcpu"
"gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -66,10 +70,21 @@ import (
"gvisor.dev/gvisor/pkg/sentry/unimpl"
uspb "gvisor.dev/gvisor/pkg/sentry/unimpl/unimplemented_syscall_go_proto"
"gvisor.dev/gvisor/pkg/sentry/uniqueid"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/state"
+ "gvisor.dev/gvisor/pkg/state/wire"
+ "gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/tcpip"
)
+// VFS2Enabled is set to true when VFS2 is enabled. Added as a global for allow
+// easy access everywhere. To be removed once VFS2 becomes the default.
+var VFS2Enabled = false
+
+// FUSEEnabled is set to true when FUSE is enabled. Added as a global for allow
+// easy access everywhere. To be removed once FUSE is completed.
+var FUSEEnabled = false
+
// Kernel represents an emulated Linux kernel. It must be initialized by calling
// Init() or LoadFrom().
//
@@ -104,7 +119,7 @@ type Kernel struct {
timekeeper *Timekeeper
tasks *TaskSet
rootUserNamespace *auth.UserNamespace
- networkStack inet.Stack `state:"nosave"`
+ rootNetworkNamespace *inet.Namespace
applicationCores uint
useHostCores bool
extraAuxv []arch.AuxEntry
@@ -183,11 +198,6 @@ type Kernel struct {
// cpuClockTickerSetting is protected by runningTasksMu.
cpuClockTickerSetting ktime.Setting
- // fdMapUids is an ever-increasing counter for generating FDTable uids.
- //
- // fdMapUids is mutable, and is accessed using atomic memory operations.
- fdMapUids uint64
-
// uniqueID is used to generate unique identifiers.
//
// uniqueID is mutable, and is accessed using atomic memory operations.
@@ -234,6 +244,36 @@ type Kernel struct {
// events. This is initialized lazily on the first unimplemented
// syscall.
unimplementedSyscallEmitter eventchannel.Emitter `state:"nosave"`
+
+ // SpecialOpts contains special kernel options.
+ SpecialOpts
+
+ // VFS keeps the filesystem state used across the kernel.
+ vfs vfs.VirtualFilesystem
+
+ // hostMount is the Mount used for file descriptors that were imported
+ // from the host.
+ hostMount *vfs.Mount
+
+ // pipeMount is the Mount used for pipes created by the pipe() and pipe2()
+ // syscalls (as opposed to named pipes created by mknod()).
+ pipeMount *vfs.Mount
+
+ // shmMount is the Mount used for anonymous files created by the
+ // memfd_create() syscalls. It is analagous to Linux's shm_mnt.
+ shmMount *vfs.Mount
+
+ // socketMount is the Mount used for sockets created by the socket() and
+ // socketpair() syscalls. There are several cases where a socket dentry will
+ // not be contained in socketMount:
+ // 1. Socket files created by mknod()
+ // 2. Socket fds imported from the host (Kernel.hostMount is used for these)
+ // 3. Socket files created by binding Unix sockets to a file path
+ socketMount *vfs.Mount
+
+ // If set to true, report address space activation waits as if the task is in
+ // external wait so that the watchdog doesn't report the task stuck.
+ SleepForAddressSpaceActivation bool
}
// InitKernelArgs holds arguments to Init.
@@ -247,8 +287,9 @@ type InitKernelArgs struct {
// RootUserNamespace is the root user namespace.
RootUserNamespace *auth.UserNamespace
- // NetworkStack is the TCP/IP network stack. NetworkStack may be nil.
- NetworkStack inet.Stack
+ // RootNetworkNamespace is the root network namespace. If nil, no networking
+ // will be available.
+ RootNetworkNamespace *inet.Namespace
// ApplicationCores is the number of logical CPUs visible to sandboxed
// applications. The set of logical CPU IDs is [0, ApplicationCores); thus
@@ -293,6 +334,9 @@ func (k *Kernel) Init(args InitKernelArgs) error {
if args.Timekeeper == nil {
return fmt.Errorf("Timekeeper is nil")
}
+ if args.Timekeeper.clocks == nil {
+ return fmt.Errorf("Must call Timekeeper.SetClocks() before Kernel.Init()")
+ }
if args.RootUserNamespace == nil {
return fmt.Errorf("RootUserNamespace is nil")
}
@@ -307,7 +351,10 @@ func (k *Kernel) Init(args InitKernelArgs) error {
k.rootUTSNamespace = args.RootUTSNamespace
k.rootIPCNamespace = args.RootIPCNamespace
k.rootAbstractSocketNamespace = args.RootAbstractSocketNamespace
- k.networkStack = args.NetworkStack
+ k.rootNetworkNamespace = args.RootNetworkNamespace
+ if k.rootNetworkNamespace == nil {
+ k.rootNetworkNamespace = inet.NewRootNamespace(nil, nil)
+ }
k.applicationCores = args.ApplicationCores
if args.UseHostCores {
k.useHostCores = true
@@ -327,13 +374,55 @@ func (k *Kernel) Init(args InitKernelArgs) error {
k.monotonicClock = &timekeeperClock{tk: args.Timekeeper, c: sentrytime.Monotonic}
k.futexes = futex.NewManager()
k.netlinkPorts = port.New()
+
+ if VFS2Enabled {
+ ctx := k.SupervisorContext()
+ if err := k.vfs.Init(ctx); err != nil {
+ return fmt.Errorf("failed to initialize VFS: %v", err)
+ }
+
+ pipeFilesystem, err := pipefs.NewFilesystem(&k.vfs)
+ if err != nil {
+ return fmt.Errorf("failed to create pipefs filesystem: %v", err)
+ }
+ defer pipeFilesystem.DecRef(ctx)
+ pipeMount, err := k.vfs.NewDisconnectedMount(pipeFilesystem, nil, &vfs.MountOptions{})
+ if err != nil {
+ return fmt.Errorf("failed to create pipefs mount: %v", err)
+ }
+ k.pipeMount = pipeMount
+
+ tmpfsFilesystem, tmpfsRoot, err := tmpfs.NewFilesystem(ctx, &k.vfs, auth.NewRootCredentials(k.rootUserNamespace))
+ if err != nil {
+ return fmt.Errorf("failed to create tmpfs filesystem: %v", err)
+ }
+ defer tmpfsFilesystem.DecRef(ctx)
+ defer tmpfsRoot.DecRef(ctx)
+ shmMount, err := k.vfs.NewDisconnectedMount(tmpfsFilesystem, tmpfsRoot, &vfs.MountOptions{})
+ if err != nil {
+ return fmt.Errorf("failed to create tmpfs mount: %v", err)
+ }
+ k.shmMount = shmMount
+
+ socketFilesystem, err := sockfs.NewFilesystem(&k.vfs)
+ if err != nil {
+ return fmt.Errorf("failed to create sockfs filesystem: %v", err)
+ }
+ defer socketFilesystem.DecRef(ctx)
+ socketMount, err := k.vfs.NewDisconnectedMount(socketFilesystem, nil, &vfs.MountOptions{})
+ if err != nil {
+ return fmt.Errorf("failed to create sockfs mount: %v", err)
+ }
+ k.socketMount = socketMount
+ }
+
return nil
}
// SaveTo saves the state of k to w.
//
// Preconditions: The kernel must be paused throughout the call to SaveTo.
-func (k *Kernel) SaveTo(w io.Writer) error {
+func (k *Kernel) SaveTo(w wire.Writer) error {
saveStart := time.Now()
ctx := k.SupervisorContext()
@@ -342,8 +431,8 @@ func (k *Kernel) SaveTo(w io.Writer) error {
defer k.extMu.Unlock()
// Stop time.
- k.pauseTimeLocked()
- defer k.resumeTimeLocked()
+ k.pauseTimeLocked(ctx)
+ defer k.resumeTimeLocked(ctx)
// Evict all evictable MemoryFile allocations.
k.mf.StartEvictions()
@@ -359,18 +448,16 @@ func (k *Kernel) SaveTo(w io.Writer) error {
// Remove all epoll waiter objects from underlying wait queues.
// NOTE: for programs to resume execution in future snapshot scenarios,
// we will need to re-establish these waiter objects after saving.
- k.tasks.unregisterEpollWaiters()
+ k.tasks.unregisterEpollWaiters(ctx)
// Clear the dirent cache before saving because Dirents must be Loaded in a
// particular order (parents before children), and Loading dirents from a cache
// breaks that order.
- if err := k.flushMountSourceRefs(); err != nil {
+ if err := k.flushMountSourceRefs(ctx); err != nil {
return err
}
- // Ensure that all pending asynchronous work is complete:
- // - inode and mount release
- // - asynchronuous IO
+ // Ensure that all inode and mount release operations have completed.
fs.AsyncBarrier()
// Once all fs work has completed (flushed references have all been released),
@@ -391,23 +478,23 @@ func (k *Kernel) SaveTo(w io.Writer) error {
//
// N.B. This will also be saved along with the full kernel save below.
cpuidStart := time.Now()
- if err := state.Save(w, k.FeatureSet(), nil); err != nil {
+ if _, err := state.Save(k.SupervisorContext(), w, k.FeatureSet()); err != nil {
return err
}
log.Infof("CPUID save took [%s].", time.Since(cpuidStart))
// Save the kernel state.
kernelStart := time.Now()
- var stats state.Stats
- if err := state.Save(w, k, &stats); err != nil {
+ stats, err := state.Save(k.SupervisorContext(), w, k)
+ if err != nil {
return err
}
- log.Infof("Kernel save stats: %s", &stats)
+ log.Infof("Kernel save stats: %s", stats.String())
log.Infof("Kernel save took [%s].", time.Since(kernelStart))
// Save the memory file's state.
memoryStart := time.Now()
- if err := k.mf.SaveTo(w); err != nil {
+ if err := k.mf.SaveTo(k.SupervisorContext(), w); err != nil {
return err
}
log.Infof("Memory save took [%s].", time.Since(memoryStart))
@@ -419,7 +506,7 @@ func (k *Kernel) SaveTo(w io.Writer) error {
// flushMountSourceRefs flushes the MountSources for all mounted filesystems
// and open FDs.
-func (k *Kernel) flushMountSourceRefs() error {
+func (k *Kernel) flushMountSourceRefs(ctx context.Context) error {
// Flush all mount sources for currently mounted filesystems in each task.
flushed := make(map[*fs.MountNamespace]struct{})
k.tasks.mu.RLock()
@@ -435,17 +522,22 @@ func (k *Kernel) flushMountSourceRefs() error {
// There may be some open FDs whose filesystems have been unmounted. We
// must flush those as well.
- return k.tasks.forEachFDPaused(func(file *fs.File) error {
+ return k.tasks.forEachFDPaused(ctx, func(file *fs.File, _ *vfs.FileDescription) error {
file.Dirent.Inode.MountSource.FlushDirentRefs()
return nil
})
}
-// forEachFDPaused applies the given function to each open file descriptor in each
-// task.
+// forEachFDPaused applies the given function to each open file descriptor in
+// each task.
//
// Precondition: Must be called with the kernel paused.
-func (ts *TaskSet) forEachFDPaused(f func(*fs.File) error) (err error) {
+func (ts *TaskSet) forEachFDPaused(ctx context.Context, f func(*fs.File, *vfs.FileDescription) error) (err error) {
+ // TODO(gvisor.dev/issue/1663): Add save support for VFS2.
+ if VFS2Enabled {
+ return nil
+ }
+
ts.mu.RLock()
defer ts.mu.RUnlock()
for t := range ts.Root.tids {
@@ -453,8 +545,8 @@ func (ts *TaskSet) forEachFDPaused(f func(*fs.File) error) (err error) {
if t.fdTable == nil {
continue
}
- t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
- if lastErr := f(file); lastErr != nil && err == nil {
+ t.fdTable.forEach(ctx, func(_ int32, file *fs.File, fileVFS2 *vfs.FileDescription, _ FDFlags) {
+ if lastErr := f(file, fileVFS2); lastErr != nil && err == nil {
err = lastErr
}
})
@@ -463,7 +555,8 @@ func (ts *TaskSet) forEachFDPaused(f func(*fs.File) error) (err error) {
}
func (ts *TaskSet) flushWritesToFiles(ctx context.Context) error {
- return ts.forEachFDPaused(func(file *fs.File) error {
+ // TODO(gvisor.dev/issue/1663): Add save support for VFS2.
+ return ts.forEachFDPaused(ctx, func(file *fs.File, _ *vfs.FileDescription) error {
if flags := file.Flags(); !flags.Write {
return nil
}
@@ -474,12 +567,9 @@ func (ts *TaskSet) flushWritesToFiles(ctx context.Context) error {
syncErr := file.Fsync(ctx, 0, fs.FileMaxOffset, fs.SyncAll)
if err := fs.SaveFileFsyncError(syncErr); err != nil {
name, _ := file.Dirent.FullName(nil /* root */)
- // Wrap this error in ErrSaveRejection
- // so that it will trigger a save
- // error, rather than a panic. This
- // also allows us to distinguish Fsync
- // errors from state file errors in
- // state.Save.
+ // Wrap this error in ErrSaveRejection so that it will trigger a save
+ // error, rather than a panic. This also allows us to distinguish Fsync
+ // errors from state file errors in state.Save.
return fs.ErrSaveRejection{
Err: fmt.Errorf("%q was not sufficiently synced: %v", name, err),
}
@@ -513,27 +603,40 @@ func (k *Kernel) invalidateUnsavableMappings(ctx context.Context) error {
return nil
}
-func (ts *TaskSet) unregisterEpollWaiters() {
+func (ts *TaskSet) unregisterEpollWaiters(ctx context.Context) {
+ // TODO(gvisor.dev/issue/1663): Add save support for VFS2.
+ if VFS2Enabled {
+ return
+ }
+
ts.mu.RLock()
defer ts.mu.RUnlock()
+
+ // Tasks that belong to the same process could potentially point to the
+ // same FDTable. So we retain a map of processed ones to avoid
+ // processing the same FDTable multiple times.
+ processed := make(map[*FDTable]struct{})
for t := range ts.Root.tids {
// We can skip locking Task.mu here since the kernel is paused.
- if t.fdTable != nil {
- t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
- if e, ok := file.FileOperations.(*epoll.EventPoll); ok {
- e.UnregisterEpollWaiters()
- }
- })
+ if t.fdTable == nil {
+ continue
}
+ if _, ok := processed[t.fdTable]; ok {
+ continue
+ }
+ t.fdTable.forEach(ctx, func(_ int32, file *fs.File, _ *vfs.FileDescription, _ FDFlags) {
+ if e, ok := file.FileOperations.(*epoll.EventPoll); ok {
+ e.UnregisterEpollWaiters()
+ }
+ })
+ processed[t.fdTable] = struct{}{}
}
}
// LoadFrom returns a new Kernel loaded from args.
-func (k *Kernel) LoadFrom(r io.Reader, net inet.Stack, clocks sentrytime.Clocks) error {
+func (k *Kernel) LoadFrom(r wire.Reader, net inet.Stack, clocks sentrytime.Clocks) error {
loadStart := time.Now()
- k.networkStack = net
-
initAppCores := k.applicationCores
// Load the pre-saved CPUID FeatureSet.
@@ -542,7 +645,7 @@ func (k *Kernel) LoadFrom(r io.Reader, net inet.Stack, clocks sentrytime.Clocks)
// don't need to explicitly install it in the Kernel.
cpuidStart := time.Now()
var features cpuid.FeatureSet
- if err := state.Load(r, &features, nil); err != nil {
+ if _, err := state.Load(k.SupervisorContext(), r, &features); err != nil {
return err
}
log.Infof("CPUID load took [%s].", time.Since(cpuidStart))
@@ -557,16 +660,20 @@ func (k *Kernel) LoadFrom(r io.Reader, net inet.Stack, clocks sentrytime.Clocks)
// Load the kernel state.
kernelStart := time.Now()
- var stats state.Stats
- if err := state.Load(r, k, &stats); err != nil {
+ stats, err := state.Load(k.SupervisorContext(), r, k)
+ if err != nil {
return err
}
- log.Infof("Kernel load stats: %s", &stats)
+ log.Infof("Kernel load stats: %s", stats.String())
log.Infof("Kernel load took [%s].", time.Since(kernelStart))
+ // rootNetworkNamespace should be populated after loading the state file.
+ // Restore the root network stack.
+ k.rootNetworkNamespace.RestoreRootStack(net)
+
// Load the memory file's state.
memoryStart := time.Now()
- if err := k.mf.LoadFrom(r); err != nil {
+ if err := k.mf.LoadFrom(k.SupervisorContext(), r); err != nil {
return err
}
log.Infof("Memory load took [%s].", time.Since(memoryStart))
@@ -622,7 +729,7 @@ type CreateProcessArgs struct {
// File is a passed host FD pointing to a file to load as the init binary.
//
// This is checked if and only if Filename is "".
- File *fs.File
+ File fsbridge.File
// Argvv is a list of arguments.
Argv []string
@@ -671,6 +778,13 @@ type CreateProcessArgs struct {
// increment it).
MountNamespace *fs.MountNamespace
+ // MountNamespaceVFS2 optionally contains the mount namespace for this
+ // process. If nil, the init process's mount namespace is used.
+ //
+ // Anyone setting MountNamespaceVFS2 must donate a reference (i.e.
+ // increment it).
+ MountNamespaceVFS2 *vfs.MountNamespace
+
// ContainerID is the container that the process belongs to.
ContainerID string
}
@@ -709,13 +823,26 @@ func (ctx *createProcessContext) Value(key interface{}) interface{} {
return ctx.args.Credentials
case fs.CtxRoot:
if ctx.args.MountNamespace != nil {
- // MountNamespace.Root() will take a reference on the root
- // dirent for us.
+ // MountNamespace.Root() will take a reference on the root dirent for us.
return ctx.args.MountNamespace.Root()
}
return nil
+ case vfs.CtxRoot:
+ if ctx.args.MountNamespaceVFS2 == nil {
+ return nil
+ }
+ // MountNamespaceVFS2.Root() takes a reference on the root dirent for us.
+ return ctx.args.MountNamespaceVFS2.Root()
+ case vfs.CtxMountNamespace:
+ if ctx.k.globalInit == nil {
+ return nil
+ }
+ // MountNamespaceVFS2 takes a reference for us.
+ return ctx.k.GlobalInit().Leader().MountNamespaceVFS2()
case fs.CtxDirentCacheLimiter:
return ctx.k.DirentCacheLimiter
+ case inet.CtxStack:
+ return ctx.k.RootNetworkNamespace().Stack()
case ktime.CtxRealtimeClock:
return ctx.k.RealtimeClock()
case limits.CtxLimits:
@@ -755,34 +882,77 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
defer k.extMu.Unlock()
log.Infof("EXEC: %v", args.Argv)
- // Grab the mount namespace.
- mounts := args.MountNamespace
- if mounts == nil {
- mounts = k.GlobalInit().Leader().MountNamespace()
- mounts.IncRef()
- }
-
- tg := k.newThreadGroup(mounts, args.PIDNamespace, NewSignalHandlers(), linux.SIGCHLD, args.Limits, k.monotonicClock)
ctx := args.NewContext(k)
- // Get the root directory from the MountNamespace.
- root := mounts.Root()
- // The call to newFSContext below will take a reference on root, so we
- // don't need to hold this one.
- defer root.DecRef()
-
- // Grab the working directory.
- remainingTraversals := uint(args.MaxSymlinkTraversals)
- wd := root // Default.
- if args.WorkingDirectory != "" {
- var err error
- wd, err = mounts.FindInode(ctx, root, nil, args.WorkingDirectory, &remainingTraversals)
- if err != nil {
- return nil, 0, fmt.Errorf("failed to find initial working directory %q: %v", args.WorkingDirectory, err)
+ var (
+ opener fsbridge.Lookup
+ fsContext *FSContext
+ mntns *fs.MountNamespace
+ )
+
+ if VFS2Enabled {
+ mntnsVFS2 := args.MountNamespaceVFS2
+ if mntnsVFS2 == nil {
+ // MountNamespaceVFS2 adds a reference to the namespace, which is
+ // transferred to the new process.
+ mntnsVFS2 = k.globalInit.Leader().MountNamespaceVFS2()
+ }
+ // Get the root directory from the MountNamespace.
+ root := args.MountNamespaceVFS2.Root()
+ // The call to newFSContext below will take a reference on root, so we
+ // don't need to hold this one.
+ defer root.DecRef(ctx)
+
+ // Grab the working directory.
+ wd := root // Default.
+ if args.WorkingDirectory != "" {
+ pop := vfs.PathOperation{
+ Root: root,
+ Start: wd,
+ Path: fspath.Parse(args.WorkingDirectory),
+ FollowFinalSymlink: true,
+ }
+ var err error
+ wd, err = k.VFS().GetDentryAt(ctx, args.Credentials, &pop, &vfs.GetDentryOptions{
+ CheckSearchable: true,
+ })
+ if err != nil {
+ return nil, 0, fmt.Errorf("failed to find initial working directory %q: %v", args.WorkingDirectory, err)
+ }
+ defer wd.DecRef(ctx)
+ }
+ opener = fsbridge.NewVFSLookup(mntnsVFS2, root, wd)
+ fsContext = NewFSContextVFS2(root, wd, args.Umask)
+
+ } else {
+ mntns = args.MountNamespace
+ if mntns == nil {
+ mntns = k.GlobalInit().Leader().MountNamespace()
+ mntns.IncRef()
+ }
+ // Get the root directory from the MountNamespace.
+ root := mntns.Root()
+ // The call to newFSContext below will take a reference on root, so we
+ // don't need to hold this one.
+ defer root.DecRef(ctx)
+
+ // Grab the working directory.
+ remainingTraversals := args.MaxSymlinkTraversals
+ wd := root // Default.
+ if args.WorkingDirectory != "" {
+ var err error
+ wd, err = mntns.FindInode(ctx, root, nil, args.WorkingDirectory, &remainingTraversals)
+ if err != nil {
+ return nil, 0, fmt.Errorf("failed to find initial working directory %q: %v", args.WorkingDirectory, err)
+ }
+ defer wd.DecRef(ctx)
}
- defer wd.DecRef()
+ opener = fsbridge.NewFSLookup(mntns, root, wd)
+ fsContext = newFSContext(root, wd, args.Umask)
}
+ tg := k.NewThreadGroup(mntns, args.PIDNamespace, NewSignalHandlers(), linux.SIGCHLD, args.Limits)
+
// Check which file to start from.
switch {
case args.Filename != "":
@@ -803,15 +973,14 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
}
// Create a fresh task context.
- remainingTraversals = uint(args.MaxSymlinkTraversals)
+ remainingTraversals := args.MaxSymlinkTraversals
loadArgs := loader.LoadArgs{
- Mounts: mounts,
- Root: root,
- WorkingDirectory: wd,
+ Opener: opener,
RemainingTraversals: &remainingTraversals,
ResolveFinal: true,
Filename: args.Filename,
File: args.File,
+ CloseOnExec: false,
Argv: args.Argv,
Envv: args.Envv,
Features: k.featureSet,
@@ -831,18 +1000,22 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
Kernel: k,
ThreadGroup: tg,
TaskContext: tc,
- FSContext: newFSContext(root, wd, args.Umask),
+ FSContext: fsContext,
FDTable: args.FDTable,
Credentials: args.Credentials,
+ NetworkNamespace: k.RootNetworkNamespace(),
AllowedCPUMask: sched.NewFullCPUSet(k.applicationCores),
UTSNamespace: args.UTSNamespace,
IPCNamespace: args.IPCNamespace,
AbstractSocketNamespace: args.AbstractSocketNamespace,
+ MountNamespaceVFS2: args.MountNamespaceVFS2,
ContainerID: args.ContainerID,
}
- if _, err := k.tasks.NewTask(config); err != nil {
+ t, err := k.tasks.NewTask(config)
+ if err != nil {
return nil, 0, err
}
+ t.traceExecEvent(tc) // Simulate exec for tracing.
// Success.
tgid := k.tasks.Root.IDOfThreadGroup(tg)
@@ -882,7 +1055,7 @@ func (k *Kernel) Start() error {
// If k was created by LoadKernelFrom, timers were stopped during
// Kernel.SaveTo and need to be resumed. If k was created by NewKernel,
// this is a no-op.
- k.resumeTimeLocked()
+ k.resumeTimeLocked(k.SupervisorContext())
// Start task goroutines.
k.tasks.mu.RLock()
defer k.tasks.mu.RUnlock()
@@ -896,7 +1069,7 @@ func (k *Kernel) Start() error {
//
// Preconditions: Any task goroutines running in k must be stopped. k.extMu
// must be locked.
-func (k *Kernel) pauseTimeLocked() {
+func (k *Kernel) pauseTimeLocked(ctx context.Context) {
// k.cpuClockTicker may be nil since Kernel.SaveTo() may be called before
// Kernel.Start().
if k.cpuClockTicker != nil {
@@ -918,9 +1091,15 @@ func (k *Kernel) pauseTimeLocked() {
// This means we'll iterate FDTables shared by multiple tasks repeatedly,
// but ktime.Timer.Pause is idempotent so this is harmless.
if t.fdTable != nil {
- t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
- if tfd, ok := file.FileOperations.(*timerfd.TimerOperations); ok {
- tfd.PauseTimer()
+ t.fdTable.forEach(ctx, func(_ int32, file *fs.File, fd *vfs.FileDescription, _ FDFlags) {
+ if VFS2Enabled {
+ if tfd, ok := fd.Impl().(*timerfd.TimerFileDescription); ok {
+ tfd.PauseTimer()
+ }
+ } else {
+ if tfd, ok := file.FileOperations.(*oldtimerfd.TimerOperations); ok {
+ tfd.PauseTimer()
+ }
}
})
}
@@ -934,7 +1113,7 @@ func (k *Kernel) pauseTimeLocked() {
//
// Preconditions: Any task goroutines running in k must be stopped. k.extMu
// must be locked.
-func (k *Kernel) resumeTimeLocked() {
+func (k *Kernel) resumeTimeLocked(ctx context.Context) {
if k.cpuClockTicker != nil {
k.cpuClockTicker.Resume()
}
@@ -948,9 +1127,15 @@ func (k *Kernel) resumeTimeLocked() {
}
}
if t.fdTable != nil {
- t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
- if tfd, ok := file.FileOperations.(*timerfd.TimerOperations); ok {
- tfd.ResumeTimer()
+ t.fdTable.forEach(ctx, func(_ int32, file *fs.File, fd *vfs.FileDescription, _ FDFlags) {
+ if VFS2Enabled {
+ if tfd, ok := fd.Impl().(*timerfd.TimerFileDescription); ok {
+ tfd.ResumeTimer()
+ }
+ } else {
+ if tfd, ok := file.FileOperations.(*oldtimerfd.TimerOperations); ok {
+ tfd.ResumeTimer()
+ }
}
})
}
@@ -1067,13 +1252,22 @@ func (k *Kernel) Kill(es ExitStatus) {
}
// Pause requests that all tasks in k temporarily stop executing, and blocks
-// until all tasks in k have stopped. Multiple calls to Pause nest and require
-// an equal number of calls to Unpause to resume execution.
+// until all tasks and asynchronous I/O operations in k have stopped. Multiple
+// calls to Pause nest and require an equal number of calls to Unpause to
+// resume execution.
func (k *Kernel) Pause() {
k.extMu.Lock()
k.tasks.BeginExternalStop()
k.extMu.Unlock()
k.tasks.runningGoroutines.Wait()
+ k.tasks.aioGoroutines.Wait()
+}
+
+// ReceiveTaskStates receives full states for all tasks.
+func (k *Kernel) ReceiveTaskStates() {
+ k.extMu.Lock()
+ k.tasks.PullFullState()
+ k.extMu.Unlock()
}
// Unpause ends the effect of a previous call to Pause. If Unpause is called
@@ -1095,6 +1289,14 @@ func (k *Kernel) SendExternalSignal(info *arch.SignalInfo, context string) {
k.sendExternalSignal(info, context)
}
+// SendExternalSignalThreadGroup injects a signal into an specific ThreadGroup.
+// This function doesn't skip signals like SendExternalSignal does.
+func (k *Kernel) SendExternalSignalThreadGroup(tg *ThreadGroup, info *arch.SignalInfo) error {
+ k.extMu.Lock()
+ defer k.extMu.Unlock()
+ return tg.SendSignal(info)
+}
+
// SendContainerSignal sends the given signal to all processes inside the
// namespace that match the given container ID.
func (k *Kernel) SendContainerSignal(cid string, info *arch.SignalInfo) error {
@@ -1117,6 +1319,22 @@ func (k *Kernel) SendContainerSignal(cid string, info *arch.SignalInfo) error {
return lastErr
}
+// RebuildTraceContexts rebuilds the trace context for all tasks.
+//
+// Unfortunately, if these are built while tracing is not enabled, then we will
+// not have meaningful trace data. Rebuilding here ensures that we can do so
+// after tracing has been enabled.
+func (k *Kernel) RebuildTraceContexts() {
+ k.extMu.Lock()
+ defer k.extMu.Unlock()
+ k.tasks.mu.RLock()
+ defer k.tasks.mu.RUnlock()
+
+ for t, tid := range k.tasks.Root.tids {
+ t.rebuildTraceContext(tid)
+ }
+}
+
// FeatureSet returns the FeatureSet.
func (k *Kernel) FeatureSet() *cpuid.FeatureSet {
return k.featureSet
@@ -1157,10 +1375,9 @@ func (k *Kernel) RootAbstractSocketNamespace() *AbstractSocketNamespace {
return k.rootAbstractSocketNamespace
}
-// NetworkStack returns the network stack. NetworkStack may return nil if no
-// network stack is available.
-func (k *Kernel) NetworkStack() inet.Stack {
- return k.networkStack
+// RootNetworkNamespace returns the root network namespace, always non-nil.
+func (k *Kernel) RootNetworkNamespace() *inet.Namespace {
+ return k.rootNetworkNamespace
}
// GlobalInit returns the thread group with ID 1 in the root PID namespace, or
@@ -1172,6 +1389,11 @@ func (k *Kernel) GlobalInit() *ThreadGroup {
return k.globalInit
}
+// TestOnly_SetGlobalInit sets the thread group with ID 1 in the root PID namespace.
+func (k *Kernel) TestOnly_SetGlobalInit(tg *ThreadGroup) {
+ k.globalInit = tg
+}
+
// ApplicationCores returns the number of CPUs visible to sandboxed
// applications.
func (k *Kernel) ApplicationCores() uint {
@@ -1255,6 +1477,11 @@ func (k *Kernel) NowMonotonic() int64 {
return now
}
+// AfterFunc implements tcpip.Clock.AfterFunc.
+func (k *Kernel) AfterFunc(d time.Duration, f func()) tcpip.Timer {
+ return ktime.TcpipAfterFunc(k.realtimeClock, d, f)
+}
+
// SetMemoryFile sets Kernel.mf. SetMemoryFile must be called before Init or
// LoadFrom.
func (k *Kernel) SetMemoryFile(mf *pgalloc.MemoryFile) {
@@ -1285,13 +1512,14 @@ func (k *Kernel) SupervisorContext() context.Context {
// +stateify savable
type SocketEntry struct {
socketEntry
- k *Kernel
- Sock *refs.WeakRef
- ID uint64 // Socket table entry number.
+ k *Kernel
+ Sock *refs.WeakRef
+ SockVFS2 *vfs.FileDescription
+ ID uint64 // Socket table entry number.
}
// WeakRefGone implements refs.WeakRefUser.WeakRefGone.
-func (s *SocketEntry) WeakRefGone() {
+func (s *SocketEntry) WeakRefGone(context.Context) {
s.k.extMu.Lock()
s.k.sockets.Remove(s)
s.k.extMu.Unlock()
@@ -1310,7 +1538,30 @@ func (k *Kernel) RecordSocket(sock *fs.File) {
k.extMu.Unlock()
}
+// RecordSocketVFS2 adds a VFS2 socket to the system-wide socket table for
+// tracking.
+//
+// Precondition: Caller must hold a reference to sock.
+//
+// Note that the socket table will not hold a reference on the
+// vfs.FileDescription, because we do not support weak refs on VFS2 files.
+func (k *Kernel) RecordSocketVFS2(sock *vfs.FileDescription) {
+ k.extMu.Lock()
+ id := k.nextSocketEntry
+ k.nextSocketEntry++
+ s := &SocketEntry{
+ k: k,
+ ID: id,
+ SockVFS2: sock,
+ }
+ k.sockets.PushBack(s)
+ k.extMu.Unlock()
+}
+
// ListSockets returns a snapshot of all sockets.
+//
+// Callers of ListSockets() in VFS2 should use SocketEntry.SockVFS2.TryIncRef()
+// to get a reference on a socket in the table.
func (k *Kernel) ListSockets() []*SocketEntry {
k.extMu.Lock()
var socks []*SocketEntry
@@ -1321,6 +1572,7 @@ func (k *Kernel) ListSockets() []*SocketEntry {
return socks
}
+// supervisorContext is a privileged context.
type supervisorContext struct {
context.NoopSleeper
log.Logger
@@ -1351,8 +1603,24 @@ func (ctx supervisorContext) Value(key interface{}) interface{} {
return ctx.k.globalInit.mounts.Root()
}
return nil
+ case vfs.CtxRoot:
+ if ctx.k.globalInit == nil {
+ return vfs.VirtualDentry{}
+ }
+ mntns := ctx.k.GlobalInit().Leader().MountNamespaceVFS2()
+ defer mntns.DecRef(ctx)
+ // Root() takes a reference on the root dirent for us.
+ return mntns.Root()
+ case vfs.CtxMountNamespace:
+ if ctx.k.globalInit == nil {
+ return nil
+ }
+ // MountNamespaceVFS2() takes a reference for us.
+ return ctx.k.GlobalInit().Leader().MountNamespaceVFS2()
case fs.CtxDirentCacheLimiter:
return ctx.k.DirentCacheLimiter
+ case inet.CtxStack:
+ return ctx.k.RootNetworkNamespace().Stack()
case ktime.CtxRealtimeClock:
return ctx.k.RealtimeClock()
case limits.CtxLimits:
@@ -1396,3 +1664,36 @@ func (k *Kernel) EmitUnimplementedEvent(ctx context.Context) {
Registers: t.Arch().StateData().Proto(),
})
}
+
+// VFS returns the virtual filesystem for the kernel.
+func (k *Kernel) VFS() *vfs.VirtualFilesystem {
+ return &k.vfs
+}
+
+// SetHostMount sets the hostfs mount.
+func (k *Kernel) SetHostMount(mnt *vfs.Mount) {
+ if k.hostMount != nil {
+ panic("Kernel.hostMount cannot be set more than once")
+ }
+ k.hostMount = mnt
+}
+
+// HostMount returns the hostfs mount.
+func (k *Kernel) HostMount() *vfs.Mount {
+ return k.hostMount
+}
+
+// PipeMount returns the pipefs mount.
+func (k *Kernel) PipeMount() *vfs.Mount {
+ return k.pipeMount
+}
+
+// ShmMount returns the tmpfs mount.
+func (k *Kernel) ShmMount() *vfs.Mount {
+ return k.shmMount
+}
+
+// SocketMount returns the sockfs mount.
+func (k *Kernel) SocketMount() *vfs.Mount {
+ return k.socketMount
+}