diff options
author | Jamie Liu <jamieliu@google.com> | 2018-08-31 13:57:02 -0700 |
---|---|---|
committer | Shentubot <shentubot@google.com> | 2018-08-31 13:58:04 -0700 |
commit | 098046ba193b839d69c059f7a0e68c89409b4237 (patch) | |
tree | 22a6bfcdbb4d440ef067c8df4643ef26f7255be1 | |
parent | b1c1afa3ccc499df3fd15814d2b6cf9005bc2ab1 (diff) |
Disintegrate kernel.TaskResources.
This allows us to call kernel.FDMap.DecRef without holding mutexes
cleanly.
PiperOrigin-RevId: 211139657
Change-Id: Ie59d5210fb9282e1950e2e40323df7264a01bcec
-rw-r--r-- | pkg/sentry/kernel/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/kernel/kernel.go | 35 | ||||
-rw-r--r-- | pkg/sentry/kernel/ptrace.go | 4 | ||||
-rw-r--r-- | pkg/sentry/kernel/task.go | 131 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_clone.go | 64 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_exec.go | 2 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_exit.go | 13 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_log.go | 2 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_resources.go | 132 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_signals.go | 24 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_start.go | 75 |
11 files changed, 230 insertions, 253 deletions
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index a7b847e94..0bc735550 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -95,7 +95,6 @@ go_library( "task_list.go", "task_log.go", "task_net.go", - "task_resources.go", "task_run.go", "task_sched.go", "task_signals.go", diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 33cd727c6..c2b5c7269 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -332,7 +332,8 @@ func (ts *TaskSet) flushWritesToFiles(ctx context.Context) error { ts.mu.RLock() defer ts.mu.RUnlock() for t := range ts.Root.tids { - if fdmap := t.FDMap(); fdmap != nil { + // We can skip locking Task.mu here since the kernel is paused. + if fdmap := t.fds; fdmap != nil { for _, desc := range fdmap.files { if flags := desc.file.Flags(); !flags.Write { continue @@ -381,7 +382,8 @@ func (ts *TaskSet) unregisterEpollWaiters() { ts.mu.RLock() defer ts.mu.RUnlock() for t := range ts.Root.tids { - if fdmap := t.FDMap(); fdmap != nil { + // We can skip locking Task.mu here since the kernel is paused. + if fdmap := t.fds; fdmap != nil { for _, desc := range fdmap.files { if desc.file != nil { if e, ok := desc.file.FileOperations.(*epoll.EventPoll); ok { @@ -625,20 +627,23 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, error) { if err != nil { return nil, err } - tr := newTaskResources(args.FDMap, newFSContext(root, wd, args.Umask)) - // NewTask unconditionally takes ownership of tr, so we never have to call - // tr.release. + + // Take a reference on the FDMap, which will be transferred to + // TaskSet.NewTask(). + args.FDMap.IncRef() // Create the task. config := &TaskConfig{ - Kernel: k, - ThreadGroup: tg, - TaskContext: tc, - TaskResources: tr, - Credentials: args.Credentials, - UTSNamespace: args.UTSNamespace, - IPCNamespace: args.IPCNamespace, - AllowedCPUMask: sched.NewFullCPUSet(k.applicationCores), + Kernel: k, + ThreadGroup: tg, + TaskContext: tc, + FSContext: newFSContext(root, wd, args.Umask), + FDMap: args.FDMap, + Credentials: args.Credentials, + AllowedCPUMask: sched.NewFullCPUSet(k.applicationCores), + UTSNamespace: args.UTSNamespace, + IPCNamespace: args.IPCNamespace, + AbstractSocketNamespace: NewAbstractSocketNamespace(), // FIXME } t, err := k.tasks.NewTask(config) if err != nil { @@ -714,7 +719,7 @@ func (k *Kernel) pauseTimeLocked() { for _, it := range t.tg.timers { it.PauseTimer() } - if fdm := t.tr.FDMap; fdm != nil { + if fdm := t.fds; fdm != nil { for _, desc := range fdm.files { if tfd, ok := desc.file.FileOperations.(*timerfd.TimerOperations); ok { tfd.PauseTimer() @@ -744,7 +749,7 @@ func (k *Kernel) resumeTimeLocked() { for _, it := range t.tg.timers { it.ResumeTimer() } - if fdm := t.tr.FDMap; fdm != nil { + if fdm := t.fds; fdm != nil { for _, desc := range fdm.files { if tfd, ok := desc.file.FileOperations.(*timerfd.TimerOperations); ok { tfd.ResumeTimer() diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go index 1a0d1876d..e21a25ae6 100644 --- a/pkg/sentry/kernel/ptrace.go +++ b/pkg/sentry/kernel/ptrace.go @@ -992,9 +992,7 @@ func (t *Task) Ptrace(req int64, pid ThreadID, addr, data usermem.Addr) error { if addr != linux.SignalSetSize { return syserror.EINVAL } - target.mu.Lock() - defer target.mu.Unlock() - _, err := t.CopyOut(data, target.tr.SignalMask) + _, err := t.CopyOut(data, target.SignalMask()) return err case linux.PTRACE_SETSIGMASK: diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index 21be3120e..32db0bf48 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -99,6 +99,19 @@ type Task struct { // ThreadGroup.signalHandlers. pendingSignals pendingSignals + // signalMask is the set of signals whose delivery is currently blocked. + // + // signalMask is accessed using atomic memory operations, and is protected + // by the signal mutex (such that reading signalMask is safe if either the + // signal mutex is locked or if atomic memory operations are used, while + // writing signalMask requires both). signalMask is owned by the task + // goroutine. + signalMask linux.SignalSet + + // FIXME: An equivalent to task_struct::real_blocked is needed + // to prevent signals that are ignored, but transiently unblocked by + // sigtimedwait(2), from being dropped in Task.sendSignalTimerLocked. + // If haveSavedSignalMask is true, savedSignalMask is the signal mask that // should be applied after the task has either delivered one signal to a // user handler or is about to resume execution in the untrusted @@ -182,25 +195,30 @@ type Task struct { // syscallRestartBlock is exclusive to the task goroutine. syscallRestartBlock SyscallRestartBlock + // p provides the mechanism by which the task runs code in userspace. The p + // interface object is immutable. + p platform.Context `state:"nosave"` + + // k is the Kernel that this task belongs to. The k pointer is immutable. + k *Kernel + // mu protects some of the following fields. mu sync.Mutex `state:"nosave"` - // tc and tr form the majority of the task's data. + // tc holds task data provided by the ELF loader. // - // tc and tr are protected by mu. tc and tr are owned by the task - // goroutine. tr.signalMask is protected by the signal mutex and must be - // written using atomic memory operations (such that reading tr.signalMask - // is safe if the signal mutex is locked or if atomic memory operations are - // used), but is also owned by the task goroutine. + // tc is protected by mu, and is owned by the task goroutine. tc TaskContext - tr TaskResources - // p provides the mechanism by which the task runs code in userspace. The p - // interface object is immutable. - p platform.Context `state:"nosave"` + // fsc is the task's filesystem context. + // + // fsc is protected by mu, and is owned by the task goroutine. + fsc *FSContext - // k is the Kernel that this task belongs to. The k pointer is immutable. - k *Kernel + // fds is the task's file descriptor table. + // + // fds is protected by mu, and is owned by the task goroutine. + fds *FDMap // If vforkParent is not nil, it is the task that created this task with // vfork() or clone(CLONE_VFORK), and should have its vforkStop ended when @@ -351,6 +369,11 @@ type Task struct { // ipcns is protected by mu. ipcns *IPCNamespace + // abstractSockets tracks abstract sockets that are in use. + // + // abstractSockets is protected by mu. + abstractSockets *AbstractSocketNamespace + // parentDeathSignal is sent to this task's thread group when its parent exits. // // parentDeathSignal is protected by mu. @@ -518,29 +541,6 @@ func (t *Task) FutexWaiter() *futex.Waiter { return t.futexWaiter } -// TaskContext returns t's TaskContext. -// -// Precondition: The caller must be running on the task goroutine, or t.mu must -// be locked. -func (t *Task) TaskContext() *TaskContext { - return &t.tc -} - -// TaskResources returns t's TaskResources. -// -// Precondition: The caller must be running on the task goroutine, or t.mu must -// be locked. -func (t *Task) TaskResources() *TaskResources { - return &t.tr -} - -// WithMuLocked executes f with t.mu locked. -func (t *Task) WithMuLocked(f func(*Task)) { - t.mu.Lock() - defer t.mu.Unlock() - f(t) -} - // Kernel returns the Kernel containing t. func (t *Task) Kernel() *Kernel { return t.k @@ -572,7 +572,7 @@ func (t *Task) Value(key interface{}) interface{} { case context.CtxThreadGroupID: return int32(t.ThreadGroup().ID()) case fs.CtxRoot: - return t.FSContext().RootDirectory() + return t.fsc.RootDirectory() case inet.CtxStack: return t.NetworkContext() case ktime.CtxRealtimeClock: @@ -619,3 +619,62 @@ func (t *Task) SyscallRestartBlock() SyscallRestartBlock { t.syscallRestartBlock = nil return r } + +// IsChrooted returns true if the root directory of t's FSContext is not the +// root directory of t's MountNamespace. +// +// Preconditions: The caller must be running on the task goroutine, or t.mu +// must be locked. +func (t *Task) IsChrooted() bool { + realRoot := t.k.mounts.Root() + defer realRoot.DecRef() + root := t.fsc.RootDirectory() + if root != nil { + defer root.DecRef() + } + return root != realRoot +} + +// TaskContext returns t's TaskContext. +// +// Precondition: The caller must be running on the task goroutine, or t.mu must +// be locked. +func (t *Task) TaskContext() *TaskContext { + return &t.tc +} + +// FSContext returns t's FSContext. FSContext does not take an additional +// reference on the returned FSContext. +// +// Precondition: The caller must be running on the task goroutine, or t.mu must +// be locked. +func (t *Task) FSContext() *FSContext { + return t.fsc +} + +// FDMap returns t's FDMap. FDMap does not take an additional reference on the +// returned FDMap. +// +// Precondition: The caller must be running on the task goroutine, or t.mu must +// be locked. +func (t *Task) FDMap() *FDMap { + return t.fds +} + +// WithMuLocked executes f with t.mu locked. +func (t *Task) WithMuLocked(f func(*Task)) { + t.mu.Lock() + defer t.mu.Unlock() + f(t) +} + +// MountNamespace returns t's MountNamespace. MountNamespace does not take an +// additional reference on the returned MountNamespace. +func (t *Task) MountNamespace() *fs.MountNamespace { + return t.k.mounts +} + +// AbstractSockets returns t's AbstractSocketNamespace. +func (t *Task) AbstractSockets() *AbstractSocketNamespace { + return t.abstractSockets +} diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go index 526165af0..46c688b20 100644 --- a/pkg/sentry/kernel/task_clone.go +++ b/pkg/sentry/kernel/task_clone.go @@ -213,6 +213,22 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) { tc.Arch.StateData().Regs.Fs_base = uint64(opts.TLS) } + var fsc *FSContext + if opts.NewFSContext { + fsc = t.fsc.Fork() + } else { + fsc = t.fsc + fsc.IncRef() + } + + var fds *FDMap + if opts.NewFiles { + fds = t.fds.Fork() + } else { + fds = t.fds + fds.IncRef() + } + pidns := t.tg.pidns if t.childPIDNamespace != nil { pidns = t.childPIDNamespace @@ -227,17 +243,21 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) { } tg = NewThreadGroup(pidns, sh, opts.TerminationSignal, tg.limits.GetCopy(), t.k.monotonicClock) } + cfg := &TaskConfig{ - Kernel: t.k, - ThreadGroup: tg, - TaskContext: tc, - TaskResources: t.tr.Fork(!opts.NewFiles, !opts.NewFSContext), - Niceness: t.Niceness(), - Credentials: creds.Fork(), - NetworkNamespaced: t.netns, - AllowedCPUMask: t.CPUMask(), - UTSNamespace: utsns, - IPCNamespace: ipcns, + Kernel: t.k, + ThreadGroup: tg, + SignalMask: t.SignalMask(), + TaskContext: tc, + FSContext: fsc, + FDMap: fds, + Credentials: creds.Fork(), + Niceness: t.Niceness(), + NetworkNamespaced: t.netns, + AllowedCPUMask: t.CPUMask(), + UTSNamespace: utsns, + IPCNamespace: ipcns, + AbstractSocketNamespace: t.abstractSockets, } if opts.NewThreadGroup { cfg.Parent = t @@ -435,15 +455,17 @@ func (t *Task) Unshare(opts *SharingOptions) error { t.childPIDNamespace = t.tg.pidns.NewChild(t.UserNamespace()) } t.mu.Lock() - defer t.mu.Unlock() + // Can't defer unlock: DecRefs must occur without holding t.mu. if opts.NewNetworkNamespace { if !haveCapSysAdmin { + t.mu.Unlock() return syserror.EPERM } t.netns = true } if opts.NewUTSNamespace { if !haveCapSysAdmin { + t.mu.Unlock() return syserror.EPERM } // Note that this must happen after NewUserNamespace, so the @@ -452,21 +474,29 @@ func (t *Task) Unshare(opts *SharingOptions) error { } if opts.NewIPCNamespace { if !haveCapSysAdmin { + t.mu.Unlock() return syserror.EPERM } // Note that "If CLONE_NEWIPC is set, then create the process in a new IPC // namespace" t.ipcns = NewIPCNamespace(t.creds.UserNamespace) } + var oldfds *FDMap if opts.NewFiles { - oldFDMap := t.tr.FDMap - t.tr.FDMap = oldFDMap.Fork() - oldFDMap.DecRef() + oldfds = t.fds + t.fds = oldfds.Fork() } + var oldfsc *FSContext if opts.NewFSContext { - oldFS := t.tr.FSContext - t.tr.FSContext = oldFS.Fork() - oldFS.DecRef() + oldfsc = t.fsc + t.fsc = oldfsc.Fork() + } + t.mu.Unlock() + if oldfds != nil { + oldfds.DecRef() + } + if oldfsc != nil { + oldfsc.DecRef() } return nil } diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go index bb3d0bd02..1b760aba4 100644 --- a/pkg/sentry/kernel/task_exec.go +++ b/pkg/sentry/kernel/task_exec.go @@ -194,7 +194,7 @@ func (r *runSyscallAfterExecStop) execute(t *Task) taskRunState { t.tg.pidns.owner.mu.Unlock() // Remove FDs with the CloseOnExec flag set. - t.FDMap().RemoveIf(func(file *fs.File, flags FDFlags) bool { + t.fds.RemoveIf(func(file *fs.File, flags FDFlags) bool { return flags.CloseOnExec }) diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index b37fcf4c1..a1b24e1c6 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -253,21 +253,22 @@ func (*runExitMain) execute(t *Task) taskRunState { } } - // Deactivate the address space before releasing the MM. + // Deactivate the address space and update max RSS before releasing the + // task's MM. t.Deactivate() - - // Update the max resident set size before releasing t.tc.mm. t.tg.pidns.owner.mu.Lock() t.updateRSSLocked() t.tg.pidns.owner.mu.Unlock() - - // Release all of the task's resources. t.mu.Lock() t.tc.release() - t.tr.release() t.mu.Unlock() + + // Releasing the MM unblocks a blocked CLONE_VFORK parent. t.unstopVforkParent() + t.fsc.DecRef() + t.fds.DecRef() + // If this is the last task to exit from the thread group, release the // thread group's resources. if lastExiter { diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go index 18efacb19..1769da210 100644 --- a/pkg/sentry/kernel/task_log.go +++ b/pkg/sentry/kernel/task_log.go @@ -63,7 +63,7 @@ func (t *Task) DebugDumpState() { if mm := t.MemoryManager(); mm != nil { t.Debugf("Mappings:\n%s", mm) } - t.Debugf("FDMap:\n%s", t.FDMap()) + t.Debugf("FDMap:\n%s", t.fds) } // debugDumpRegisters logs register state at log level debug. diff --git a/pkg/sentry/kernel/task_resources.go b/pkg/sentry/kernel/task_resources.go deleted file mode 100644 index 0832bf989..000000000 --- a/pkg/sentry/kernel/task_resources.go +++ /dev/null @@ -1,132 +0,0 @@ -// Copyright 2018 Google Inc. -// -// 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 kernel - -import ( - "gvisor.googlesource.com/gvisor/pkg/abi/linux" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" -) - -// TaskResources is the subset of a task's data provided by its creator that is -// not provided by the loader. -// -// +stateify savable -type TaskResources struct { - // SignalMask is the set of signals whose delivery is currently blocked. - // - // FIXME: Determine if we also need RealSignalMask - SignalMask linux.SignalSet - - // FSContext is the filesystem context. - *FSContext - - // FDMap provides access to files to the task. - *FDMap - - // Tracks abstract sockets that are in use. - AbstractSockets *AbstractSocketNamespace -} - -// newTaskResources returns a new TaskResources, taking an additional reference -// on fdm. -func newTaskResources(fdm *FDMap, fc *FSContext) *TaskResources { - fdm.IncRef() - return &TaskResources{ - FDMap: fdm, - FSContext: fc, - AbstractSockets: NewAbstractSocketNamespace(), - } -} - -// release releases all resources held by the TaskResources. release is called -// by the task when it exits. -func (tr *TaskResources) release() { - tr.FDMap.DecRef() - tr.FDMap = nil - tr.FSContext.DecRef() - tr.FSContext = nil - tr.AbstractSockets = nil -} - -// Fork returns a duplicate of tr. -// -// FIXME: Preconditions: When tr is owned by a Task, that task's -// signal mutex must be locked, or Fork must be called by the task's goroutine. -func (tr *TaskResources) Fork(shareFiles bool, shareFSContext bool) *TaskResources { - var fdmap *FDMap - if shareFiles { - fdmap = tr.FDMap - fdmap.IncRef() - } else { - fdmap = tr.FDMap.Fork() - } - - var fsc *FSContext - if shareFSContext { - fsc = tr.FSContext - fsc.IncRef() - } else { - fsc = tr.FSContext.Fork() - } - - return &TaskResources{ - SignalMask: tr.SignalMask, - FDMap: fdmap, - FSContext: fsc, - AbstractSockets: tr.AbstractSockets, - } -} - -// FDMap returns t's FDMap. -// -// Preconditions: The caller must be running on the task goroutine, or t.mu -// must be locked. -func (t *Task) FDMap() *FDMap { - return t.tr.FDMap -} - -// FSContext returns t's FSContext. -// -// Preconditions: The caller must be running on the task goroutine, or t.mu -// must be locked. -func (t *Task) FSContext() *FSContext { - return t.tr.FSContext -} - -// MountNamespace returns t's MountNamespace. MountNamespace does not take an additional -// reference on the returned MountNamespace. -func (t *Task) MountNamespace() *fs.MountNamespace { - return t.k.mounts -} - -// AbstractSockets returns t's AbstractSocketNamespace. -func (t *Task) AbstractSockets() *AbstractSocketNamespace { - return t.tr.AbstractSockets -} - -// IsChrooted returns true if the root directory of t's FSContext is not the -// root directory of t's MountNamespace. -// -// Preconditions: The caller must be running on the task goroutine, or t.mu -// must be locked. -func (t *Task) IsChrooted() bool { - realRoot := t.k.mounts.Root() - defer realRoot.DecRef() - root := t.tr.FSContext.RootDirectory() - if root != nil { - defer root.DecRef() - } - return root != realRoot -} diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go index 4a66bce6b..58a1bc0bd 100644 --- a/pkg/sentry/kernel/task_signals.go +++ b/pkg/sentry/kernel/task_signals.go @@ -124,10 +124,10 @@ var StopSignals = linux.MakeSignalSet(linux.SIGSTOP, linux.SIGTSTP, linux.SIGTTI // // Preconditions: t.tg.signalHandlers.mu must be locked. func (t *Task) dequeueSignalLocked() *arch.SignalInfo { - if info := t.pendingSignals.dequeue(t.tr.SignalMask); info != nil { + if info := t.pendingSignals.dequeue(t.signalMask); info != nil { return info } - return t.tg.pendingSignals.dequeue(t.tr.SignalMask) + return t.tg.pendingSignals.dequeue(t.signalMask) } // TakeSignal returns a pending signal not blocked by mask. Signal handlers are @@ -252,7 +252,7 @@ func (t *Task) deliverSignalToHandler(info *arch.SignalInfo, act arch.SignalAct) // handler should run with the current mask, but sigreturn should restore // the saved one. st := &arch.Stack{t.Arch(), t.MemoryManager(), sp} - mask := t.tr.SignalMask + mask := t.signalMask if t.haveSavedSignalMask { mask = t.savedSignalMask } @@ -262,7 +262,7 @@ func (t *Task) deliverSignalToHandler(info *arch.SignalInfo, act arch.SignalAct) t.haveSavedSignalMask = false // Add our signal mask. - newMask := t.tr.SignalMask | act.Mask + newMask := t.signalMask | act.Mask if !act.IsNoDefer() { newMask |= linux.SignalSetOf(linux.Signal(info.Signo)) } @@ -431,7 +431,7 @@ func (t *Task) sendSignalTimerLocked(info *arch.SignalInfo, group bool, timer *I // Linux's kernel/signal.c:__send_signal() => prepare_signal() => // sig_ignored(). ignored := computeAction(sig, t.tg.signalHandlers.actions[sig]) == SignalActionIgnore - if linux.SignalSetOf(sig)&t.tr.SignalMask == 0 && ignored && !t.hasTracer() { + if linux.SignalSetOf(sig)&t.signalMask == 0 && ignored && !t.hasTracer() { t.Debugf("Discarding ignored signal %d", sig) if timer != nil { timer.signalRejectedLocked() @@ -515,7 +515,7 @@ func (tg *ThreadGroup) applySignalSideEffectsLocked(sig linux.Signal) { // Preconditions: The signal mutex must be locked. func (t *Task) canReceiveSignalLocked(sig linux.Signal) bool { // - Do not choose tasks that are blocking the signal. - if linux.SignalSetOf(sig)&t.tr.SignalMask != 0 { + if linux.SignalSetOf(sig)&t.signalMask != 0 { return false } // - No need to check Task.exitState, as the exit path sets every bit in the @@ -564,21 +564,21 @@ func (t *Task) forceSignal(sig linux.Signal, unconditional bool) { } func (t *Task) forceSignalLocked(sig linux.Signal, unconditional bool) { - blocked := linux.SignalSetOf(sig)&t.tr.SignalMask != 0 + blocked := linux.SignalSetOf(sig)&t.signalMask != 0 act := t.tg.signalHandlers.actions[sig] ignored := act.Handler == arch.SignalActIgnore if blocked || ignored || unconditional { act.Handler = arch.SignalActDefault t.tg.signalHandlers.actions[sig] = act if blocked { - t.setSignalMaskLocked(t.tr.SignalMask &^ linux.SignalSetOf(sig)) + t.setSignalMaskLocked(t.signalMask &^ linux.SignalSetOf(sig)) } } } // SignalMask returns a copy of t's signal mask. func (t *Task) SignalMask() linux.SignalSet { - return linux.SignalSet(atomic.LoadUint64((*uint64)(&t.tr.SignalMask))) + return linux.SignalSet(atomic.LoadUint64((*uint64)(&t.signalMask))) } // SetSignalMask sets t's signal mask. @@ -595,8 +595,8 @@ func (t *Task) SetSignalMask(mask linux.SignalSet) { // Preconditions: The signal mutex must be locked. func (t *Task) setSignalMaskLocked(mask linux.SignalSet) { - oldMask := t.tr.SignalMask - atomic.StoreUint64((*uint64)(&t.tr.SignalMask), uint64(mask)) + oldMask := t.signalMask + atomic.StoreUint64((*uint64)(&t.signalMask), uint64(mask)) // If the new mask blocks any signals that were not blocked by the old // mask, and at least one such signal is pending in tg.pendingSignals, and @@ -1076,7 +1076,7 @@ func (*runInterruptAfterSignalDeliveryStop) execute(t *Task) taskRunState { t.tg.signalHandlers.mu.Lock() t.tg.pidns.owner.mu.Unlock() // If the signal is masked, re-queue it. - if linux.SignalSetOf(sig)&t.tr.SignalMask != 0 { + if linux.SignalSetOf(sig)&t.signalMask != 0 { t.sendSignalLocked(info, false /* group */) t.tg.signalHandlers.mu.Unlock() return (*runInterrupt)(nil) diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go index c97dee8fc..6ce99d268 100644 --- a/pkg/sentry/kernel/task_start.go +++ b/pkg/sentry/kernel/task_start.go @@ -15,6 +15,7 @@ package kernel import ( + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/futex" @@ -26,7 +27,7 @@ import ( // TaskConfig defines the configuration of a new Task (see below). type TaskConfig struct { // Kernel is the owning Kernel. - *Kernel + Kernel *Kernel // Parent is the new task's parent. Parent may be nil. Parent *Task @@ -36,13 +37,24 @@ type TaskConfig struct { InheritParent *Task // ThreadGroup is the ThreadGroup the new task belongs to. - *ThreadGroup + ThreadGroup *ThreadGroup - // TaskContext is the TaskContext of the new task. - *TaskContext + // SignalMask is the new task's initial signal mask. + SignalMask linux.SignalSet - // TaskResources is the TaskResources of the new task. - *TaskResources + // TaskContext is the TaskContext of the new task. Ownership of the + // TaskContext is transferred to TaskSet.NewTask, whether or not it + // succeeds. + TaskContext *TaskContext + + // FSContext is the FSContext of the new task. A reference must be held on + // FSContext, which is transferred to TaskSet.NewTask whether or not it + // succeeds. + FSContext *FSContext + + // FDMap is the FDMap of the new task. A reference must be held on FDMap, + // which is transferred to TaskSet.NewTask whether or not it succeeds. + FDMap *FDMap // Credentials is the Credentials of the new task. Credentials *auth.Credentials @@ -62,25 +74,27 @@ type TaskConfig struct { // IPCNamespace is the IPCNamespace of the new task. IPCNamespace *IPCNamespace + + // AbstractSocketNamespace is the AbstractSocketNamespace of the new task. + AbstractSocketNamespace *AbstractSocketNamespace } -// NewTask creates a new task defined by TaskConfig. -// Whether or not NewTask is successful, it takes ownership of both TaskContext -// and TaskResources of the TaskConfig. +// NewTask creates a new task defined by cfg. // // NewTask does not start the returned task; the caller must call Task.Start. func (ts *TaskSet) NewTask(cfg *TaskConfig) (*Task, error) { t, err := ts.newTask(cfg) if err != nil { cfg.TaskContext.release() - cfg.TaskResources.release() + cfg.FSContext.DecRef() + cfg.FDMap.DecRef() return nil, err } return t, nil } -// newTask is a helper for TaskSet.NewTask that only takes ownership of TaskContext -// and TaskResources of the TaskConfig if it succeeds. +// newTask is a helper for TaskSet.NewTask that only takes ownership of parts +// of cfg if it succeeds. func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) { tg := cfg.ThreadGroup tc := cfg.TaskContext @@ -90,23 +104,26 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) { parent: cfg.Parent, children: make(map[*Task]struct{}), }, - runState: (*runApp)(nil), - interruptChan: make(chan struct{}, 1), - signalStack: arch.SignalStack{Flags: arch.SignalStackFlagDisable}, - tc: *tc, - tr: *cfg.TaskResources, - p: cfg.Kernel.Platform.NewContext(), - k: cfg.Kernel, - ptraceTracees: make(map[*Task]struct{}), - allowedCPUMask: cfg.AllowedCPUMask.Copy(), - ioUsage: &usage.IO{}, - creds: cfg.Credentials, - niceness: cfg.Niceness, - netns: cfg.NetworkNamespaced, - utsns: cfg.UTSNamespace, - ipcns: cfg.IPCNamespace, - rseqCPU: -1, - futexWaiter: futex.NewWaiter(), + runState: (*runApp)(nil), + interruptChan: make(chan struct{}, 1), + signalMask: cfg.SignalMask, + signalStack: arch.SignalStack{Flags: arch.SignalStackFlagDisable}, + tc: *tc, + fsc: cfg.FSContext, + fds: cfg.FDMap, + p: cfg.Kernel.Platform.NewContext(), + k: cfg.Kernel, + ptraceTracees: make(map[*Task]struct{}), + allowedCPUMask: cfg.AllowedCPUMask.Copy(), + ioUsage: &usage.IO{}, + creds: cfg.Credentials, + niceness: cfg.Niceness, + netns: cfg.NetworkNamespaced, + utsns: cfg.UTSNamespace, + ipcns: cfg.IPCNamespace, + abstractSockets: cfg.AbstractSocketNamespace, + rseqCPU: -1, + futexWaiter: futex.NewWaiter(), } t.endStopCond.L = &t.tg.signalHandlers.mu t.ptraceTracer.Store((*Task)(nil)) |