diff options
author | Jamie Liu <jamieliu@google.com> | 2018-10-17 15:48:55 -0700 |
---|---|---|
committer | Shentubot <shentubot@google.com> | 2018-10-17 15:50:02 -0700 |
commit | b2a88ff4713325fca736f6a3bf200be02d2d72a7 (patch) | |
tree | 489e54828c2bfe0bf326976920e5d5e612f877a0 /pkg/sentry/kernel/task_sched.go | |
parent | 6922eee6499212a009fdc254224f916bd1c46f29 (diff) |
Check thread group CPU timers in the CPU clock ticker.
This reduces the number of goroutines and runtime timers when
ITIMER_VIRTUAL or ITIMER_PROF are enabled, or when RLIMIT_CPU is set.
This also ensures that thread group CPU timers only advance if running
tasks are observed at the time the CPU clock advances, mostly
eliminating the possibility that a CPU timer expiration observes no
running tasks and falls back to the group leader.
PiperOrigin-RevId: 217603396
Change-Id: Ia24ce934d5574334857d9afb5ad8ca0b6a6e65f4
Diffstat (limited to 'pkg/sentry/kernel/task_sched.go')
-rw-r--r-- | pkg/sentry/kernel/task_sched.go | 344 |
1 files changed, 325 insertions, 19 deletions
diff --git a/pkg/sentry/kernel/task_sched.go b/pkg/sentry/kernel/task_sched.go index 49141ab74..19dcc963a 100644 --- a/pkg/sentry/kernel/task_sched.go +++ b/pkg/sentry/kernel/task_sched.go @@ -18,12 +18,15 @@ package kernel import ( "fmt" + "math/rand" "sync/atomic" "time" "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/hostcpu" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/sched" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/limits" "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -84,6 +87,33 @@ type TaskGoroutineSchedInfo struct { SysTicks uint64 } +// userTicksAt returns the extrapolated value of ts.UserTicks after +// Kernel.CPUClockNow() indicates a time of now. +// +// Preconditions: now <= Kernel.CPUClockNow(). (Since Kernel.cpuClock is +// monotonic, this is satisfied if now is the result of a previous call to +// Kernel.CPUClockNow().) This requirement exists because otherwise a racing +// change to t.gosched can cause userTicksAt to adjust stats by too much, +// making the observed stats non-monotonic. +func (ts *TaskGoroutineSchedInfo) userTicksAt(now uint64) uint64 { + if ts.Timestamp < now && ts.State == TaskGoroutineRunningApp { + // Update stats to reflect execution since the last update. + return ts.UserTicks + (now - ts.Timestamp) + } + return ts.UserTicks +} + +// sysTicksAt returns the extrapolated value of ts.SysTicks after +// Kernel.CPUClockNow() indicates a time of now. +// +// Preconditions: As for userTicksAt. +func (ts *TaskGoroutineSchedInfo) sysTicksAt(now uint64) uint64 { + if ts.Timestamp < now && ts.State == TaskGoroutineRunningSys { + return ts.SysTicks + (now - ts.Timestamp) + } + return ts.SysTicks +} + // Preconditions: The caller must be running on the task goroutine. func (t *Task) accountTaskGoroutineEnter(state TaskGoroutineState) { now := t.k.CPUClockNow() @@ -127,26 +157,12 @@ func (t *Task) CPUStats() usage.CPUStats { return t.cpuStatsAt(t.k.CPUClockNow()) } -// Preconditions: now <= Kernel.CPUClockNow(). (Since Kernel.cpuClock is -// monotonic, this is satisfied if now is the result of a previous call to -// Kernel.CPUClockNow().) This requirement exists because otherwise a racing -// change to t.gosched can cause cpuStatsAt to adjust stats by too much, making -// the returned stats non-monotonic. +// Preconditions: As for TaskGoroutineSchedInfo.userTicksAt. func (t *Task) cpuStatsAt(now uint64) usage.CPUStats { tsched := t.TaskGoroutineSchedInfo() - if tsched.Timestamp < now { - // Update stats to reflect execution since the last update to - // t.gosched. - switch tsched.State { - case TaskGoroutineRunningSys: - tsched.SysTicks += now - tsched.Timestamp - case TaskGoroutineRunningApp: - tsched.UserTicks += now - tsched.Timestamp - } - } return usage.CPUStats{ - UserTime: time.Duration(tsched.UserTicks * uint64(linux.ClockTick)), - SysTime: time.Duration(tsched.SysTicks * uint64(linux.ClockTick)), + UserTime: time.Duration(tsched.userTicksAt(now) * uint64(linux.ClockTick)), + SysTime: time.Duration(tsched.sysTicksAt(now) * uint64(linux.ClockTick)), VoluntarySwitches: atomic.LoadUint64(&t.yieldCount), } } @@ -162,9 +178,14 @@ func (tg *ThreadGroup) CPUStats() usage.CPUStats { // ThreadGroup has ever executed anyway. return usage.CPUStats{} } - now := tg.leader.k.CPUClockNow() + return tg.cpuStatsAtLocked(tg.leader.k.CPUClockNow()) +} + +// Preconditions: As for TaskGoroutineSchedInfo.userTicksAt. The TaskSet mutex +// must be locked. +func (tg *ThreadGroup) cpuStatsAtLocked(now uint64) usage.CPUStats { stats := tg.exitedCPUStats - // Account for active tasks. + // Account for live tasks. for t := tg.tasks.Front(); t != nil; t = t.Next() { stats.Accumulate(t.cpuStatsAt(now)) } @@ -182,6 +203,291 @@ func (tg *ThreadGroup) JoinedChildCPUStats() usage.CPUStats { return tg.childCPUStats } +// taskClock is a ktime.Clock that measures the time that a task has spent +// executing. taskClock is primarily used to implement CLOCK_THREAD_CPUTIME_ID. +// +// +stateify savable +type taskClock struct { + t *Task + + // If includeSys is true, the taskClock includes both time spent executing + // application code as well as time spent in the sentry. Otherwise, the + // taskClock includes only time spent executing application code. + includeSys bool + + // Implements waiter.Waitable. TimeUntil wouldn't change its estimation + // based on either of the clock events, so there's no event to be + // notified for. + ktime.NoClockEvents `state:"nosave"` + + // Implements ktime.Clock.WallTimeUntil. + // + // As an upper bound, a task's clock cannot advance faster than CPU + // time. It would have to execute at a rate of more than 1 task-second + // per 1 CPU-second, which isn't possible. + ktime.WallRateClock `state:"nosave"` +} + +// UserCPUClock returns a clock measuring the CPU time the task has spent +// executing application code. +func (t *Task) UserCPUClock() ktime.Clock { + return &taskClock{t: t, includeSys: false} +} + +// CPUClock returns a clock measuring the CPU time the task has spent executing +// application and "kernel" code. +func (t *Task) CPUClock() ktime.Clock { + return &taskClock{t: t, includeSys: true} +} + +// Now implements ktime.Clock.Now. +func (tc *taskClock) Now() ktime.Time { + stats := tc.t.CPUStats() + if tc.includeSys { + return ktime.FromNanoseconds((stats.UserTime + stats.SysTime).Nanoseconds()) + } + return ktime.FromNanoseconds(stats.UserTime.Nanoseconds()) +} + +// tgClock is a ktime.Clock that measures the time a thread group has spent +// executing. tgClock is primarily used to implement CLOCK_PROCESS_CPUTIME_ID. +// +// +stateify savable +type tgClock struct { + tg *ThreadGroup + + // If includeSys is true, the tgClock includes both time spent executing + // application code as well as time spent in the sentry. Otherwise, the + // tgClock includes only time spent executing application code. + includeSys bool + + // Implements waiter.Waitable. + ktime.ClockEventsQueue `state:"nosave"` +} + +// Now implements ktime.Clock.Now. +func (tgc *tgClock) Now() ktime.Time { + stats := tgc.tg.CPUStats() + if tgc.includeSys { + return ktime.FromNanoseconds((stats.UserTime + stats.SysTime).Nanoseconds()) + } + return ktime.FromNanoseconds(stats.UserTime.Nanoseconds()) +} + +// WallTimeUntil implements ktime.Clock.WallTimeUntil. +func (tgc *tgClock) WallTimeUntil(t, now ktime.Time) time.Duration { + // Thread group CPU time should not exceed wall time * live tasks, since + // task goroutines exit after the transition to TaskExitZombie in + // runExitNotify. + tgc.tg.pidns.owner.mu.RLock() + n := tgc.tg.liveTasks + tgc.tg.pidns.owner.mu.RUnlock() + if n == 0 { + if t.Before(now) { + return 0 + } + // The timer tick raced with thread group exit, after which no more + // tasks can enter the thread group. So tgc.Now() will never advance + // again. Return a large delay; the timer should be stopped long before + // it comes again anyway. + return time.Hour + } + // This is a lower bound on the amount of time that can elapse before an + // associated timer expires, so returning this value tends to result in a + // sequence of closely-spaced ticks just before timer expiry. To avoid + // this, round up to the nearest ClockTick; CPU usage measurements are + // limited to this resolution anyway. + remaining := time.Duration(t.Sub(now).Nanoseconds()/int64(n)) * time.Nanosecond + return ((remaining + (linux.ClockTick - time.Nanosecond)) / linux.ClockTick) * linux.ClockTick +} + +// UserCPUClock returns a ktime.Clock that measures the time that a thread +// group has spent executing. +func (tg *ThreadGroup) UserCPUClock() ktime.Clock { + return &tgClock{tg: tg, includeSys: false} +} + +// CPUClock returns a ktime.Clock that measures the time that a thread group +// has spent executing, including sentry time. +func (tg *ThreadGroup) CPUClock() ktime.Clock { + return &tgClock{tg: tg, includeSys: true} +} + +type kernelCPUClockTicker struct { + k *Kernel + + // These are essentially kernelCPUClockTicker.Notify local variables that + // are cached between calls to reduce allocations. + rng *rand.Rand + tgs []*ThreadGroup +} + +func newKernelCPUClockTicker(k *Kernel) *kernelCPUClockTicker { + return &kernelCPUClockTicker{ + k: k, + rng: rand.New(rand.NewSource(rand.Int63())), + } +} + +// Notify implements ktime.TimerListener.Notify. +func (ticker *kernelCPUClockTicker) Notify(exp uint64) { + // Only increment cpuClock by 1 regardless of the number of expirations. + // This approximately compensates for cases where thread throttling or bad + // Go runtime scheduling prevents the kernelCPUClockTicker goroutine, and + // presumably task goroutines as well, from executing for a long period of + // time. It's also necessary to prevent CPU clocks from seeing large + // discontinuous jumps. + now := atomic.AddUint64(&ticker.k.cpuClock, 1) + + // Check thread group CPU timers. + tgs := ticker.k.tasks.Root.ThreadGroupsAppend(ticker.tgs) + for _, tg := range tgs { + if atomic.LoadUint32(&tg.cpuTimersEnabled) == 0 { + continue + } + + ticker.k.tasks.mu.RLock() + if tg.leader == nil { + // No tasks have ever run in this thread group. + ticker.k.tasks.mu.RUnlock() + continue + } + // Accumulate thread group CPU stats, and randomly select running tasks + // using reservoir sampling to receive CPU timer signals. + var virtReceiver *Task + nrVirtCandidates := 0 + var profReceiver *Task + nrProfCandidates := 0 + tgUserTime := tg.exitedCPUStats.UserTime + tgSysTime := tg.exitedCPUStats.SysTime + for t := tg.tasks.Front(); t != nil; t = t.Next() { + tsched := t.TaskGoroutineSchedInfo() + tgUserTime += time.Duration(tsched.userTicksAt(now) * uint64(linux.ClockTick)) + tgSysTime += time.Duration(tsched.sysTicksAt(now) * uint64(linux.ClockTick)) + switch tsched.State { + case TaskGoroutineRunningApp: + // Considered by ITIMER_VIRT, ITIMER_PROF, and RLIMIT_CPU + // timers. + nrVirtCandidates++ + if int(randInt31n(ticker.rng, int32(nrVirtCandidates))) == 0 { + virtReceiver = t + } + fallthrough + case TaskGoroutineRunningSys: + // Considered by ITIMER_PROF and RLIMIT_CPU timers. + nrProfCandidates++ + if int(randInt31n(ticker.rng, int32(nrProfCandidates))) == 0 { + profReceiver = t + } + } + } + tgVirtNow := ktime.FromNanoseconds(tgUserTime.Nanoseconds()) + tgProfNow := ktime.FromNanoseconds((tgUserTime + tgSysTime).Nanoseconds()) + + // All of the following are standard (not real-time) signals, which are + // automatically deduplicated, so we ignore the number of expirations. + tg.signalHandlers.mu.Lock() + // It should only be possible for these timers to advance if we found + // at least one running task. + if virtReceiver != nil { + // ITIMER_VIRTUAL + newItimerVirtSetting, exp := tg.itimerVirtSetting.At(tgVirtNow) + tg.itimerVirtSetting = newItimerVirtSetting + if exp != 0 { + virtReceiver.sendSignalLocked(sigPriv(linux.SIGVTALRM), true) + } + } + if profReceiver != nil { + // ITIMER_PROF + newItimerProfSetting, exp := tg.itimerProfSetting.At(tgProfNow) + tg.itimerProfSetting = newItimerProfSetting + if exp != 0 { + profReceiver.sendSignalLocked(sigPriv(linux.SIGPROF), true) + } + // RLIMIT_CPU soft limit + newRlimitCPUSoftSetting, exp := tg.rlimitCPUSoftSetting.At(tgProfNow) + tg.rlimitCPUSoftSetting = newRlimitCPUSoftSetting + if exp != 0 { + profReceiver.sendSignalLocked(sigPriv(linux.SIGXCPU), true) + } + // RLIMIT_CPU hard limit + rlimitCPUMax := tg.limits.Get(limits.CPU).Max + if rlimitCPUMax != limits.Infinity && !tgProfNow.Before(ktime.FromSeconds(int64(rlimitCPUMax))) { + profReceiver.sendSignalLocked(sigPriv(linux.SIGKILL), true) + } + } + tg.signalHandlers.mu.Unlock() + + ticker.k.tasks.mu.RUnlock() + } + + // Retain tgs between calls to Notify to reduce allocations. + for i := range tgs { + tgs[i] = nil + } + ticker.tgs = tgs[:0] +} + +// Destroy implements ktime.TimerListener.Destroy. +func (ticker *kernelCPUClockTicker) Destroy() { +} + +// randInt31n returns a random integer in [0, n). +// +// randInt31n is equivalent to math/rand.Rand.int31n(), which is unexported. +// See that function for details. +func randInt31n(rng *rand.Rand, n int32) int32 { + v := rng.Uint32() + prod := uint64(v) * uint64(n) + low := uint32(prod) + if low < uint32(n) { + thresh := uint32(-n) % uint32(n) + for low < thresh { + v = rng.Uint32() + prod = uint64(v) * uint64(n) + low = uint32(prod) + } + } + return int32(prod >> 32) +} + +// NotifyRlimitCPUUpdated is called by setrlimit. +// +// Preconditions: The caller must be running on the task goroutine. +func (t *Task) NotifyRlimitCPUUpdated() { + t.k.cpuClockTicker.Atomically(func() { + t.tg.pidns.owner.mu.RLock() + defer t.tg.pidns.owner.mu.RUnlock() + t.tg.signalHandlers.mu.Lock() + defer t.tg.signalHandlers.mu.Unlock() + rlimitCPU := t.tg.limits.Get(limits.CPU) + t.tg.rlimitCPUSoftSetting = ktime.Setting{ + Enabled: rlimitCPU.Cur != limits.Infinity, + Next: ktime.FromNanoseconds((time.Duration(rlimitCPU.Cur) * time.Second).Nanoseconds()), + Period: time.Second, + } + if rlimitCPU.Max != limits.Infinity { + // Check if tg is already over the hard limit. + tgcpu := t.tg.cpuStatsAtLocked(t.k.CPUClockNow()) + tgProfNow := ktime.FromNanoseconds((tgcpu.UserTime + tgcpu.SysTime).Nanoseconds()) + if !tgProfNow.Before(ktime.FromSeconds(int64(rlimitCPU.Max))) { + t.sendSignalLocked(sigPriv(linux.SIGKILL), true) + } + } + t.tg.updateCPUTimersEnabledLocked() + }) +} + +// Preconditions: The signal mutex must be locked. +func (tg *ThreadGroup) updateCPUTimersEnabledLocked() { + rlimitCPU := tg.limits.Get(limits.CPU) + if tg.itimerVirtSetting.Enabled || tg.itimerProfSetting.Enabled || tg.rlimitCPUSoftSetting.Enabled || rlimitCPU.Max != limits.Infinity { + atomic.StoreUint32(&tg.cpuTimersEnabled, 1) + } else { + atomic.StoreUint32(&tg.cpuTimersEnabled, 0) + } +} + // StateStatus returns a string representation of the task's current state, // appropriate for /proc/[pid]/status. func (t *Task) StateStatus() string { |