summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry
diff options
context:
space:
mode:
authorJamie Liu <jamieliu@google.com>2018-10-17 15:48:55 -0700
committerShentubot <shentubot@google.com>2018-10-17 15:50:02 -0700
commitb2a88ff4713325fca736f6a3bf200be02d2d72a7 (patch)
tree489e54828c2bfe0bf326976920e5d5e612f877a0 /pkg/sentry
parent6922eee6499212a009fdc254224f916bd1c46f29 (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')
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/kernel.go46
-rw-r--r--pkg/sentry/kernel/task_acct.go97
-rw-r--r--pkg/sentry/kernel/task_clone.go2
-rw-r--r--pkg/sentry/kernel/task_exit.go3
-rw-r--r--pkg/sentry/kernel/task_run.go4
-rw-r--r--pkg/sentry/kernel/task_sched.go344
-rw-r--r--pkg/sentry/kernel/task_signals.go66
-rw-r--r--pkg/sentry/kernel/thread_group.go73
-rw-r--r--pkg/sentry/kernel/threads.go6
-rw-r--r--pkg/sentry/kernel/time/time.go35
-rw-r--r--pkg/sentry/kernel/timekeeper.go26
-rw-r--r--pkg/sentry/kernel/timer.go290
-rw-r--r--pkg/sentry/syscalls/linux/sys_rlimit.go2
-rw-r--r--pkg/sentry/syscalls/linux/sys_timer.go61
15 files changed, 551 insertions, 505 deletions
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index acc61cb09..e2fb61ba6 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -111,7 +111,6 @@ go_library(
"threads.go",
"timekeeper.go",
"timekeeper_state.go",
- "timer.go",
"uts_namespace.go",
"vdso.go",
"version.go",
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index 84afdb530..5d6856f3c 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -20,7 +20,7 @@
//
// Kernel.extMu
// ThreadGroup.timerMu
-// ktime.Timer.mu (for IntervalTimer)
+// ktime.Timer.mu (for kernelCPUClockTicker and IntervalTimer)
// TaskSet.mu
// SignalHandlers.mu
// Task.mu
@@ -617,7 +617,7 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
return nil, 0, fmt.Errorf("no kernel MountNamespace")
}
- tg := NewThreadGroup(k.tasks.Root, NewSignalHandlers(), linux.SIGCHLD, args.Limits, k.monotonicClock)
+ tg := k.newThreadGroup(k.tasks.Root, NewSignalHandlers(), linux.SIGCHLD, args.Limits, k.monotonicClock)
ctx := args.NewContext(k)
// Grab the root directory.
@@ -705,7 +705,7 @@ func (k *Kernel) Start() error {
}
k.started = true
- k.cpuClockTicker = ktime.NewTimer(k.monotonicClock, kernelCPUClockListener{k})
+ k.cpuClockTicker = ktime.NewTimer(k.monotonicClock, newKernelCPUClockTicker(k))
k.cpuClockTicker.Swap(ktime.Setting{
Enabled: true,
Period: linux.ClockTick,
@@ -741,14 +741,13 @@ func (k *Kernel) pauseTimeLocked() {
// mutex, while holding the Timer mutex.)
for t := range k.tasks.Root.tids {
if t == t.tg.leader {
- t.tg.tm.pause()
- }
- // This means we'll iterate ThreadGroups and FDMaps shared by multiple
- // tasks repeatedly, but ktime.Timer.Pause is idempotent so this is
- // harmless.
- for _, it := range t.tg.timers {
- it.PauseTimer()
+ t.tg.itimerRealTimer.Pause()
+ for _, it := range t.tg.timers {
+ it.PauseTimer()
+ }
}
+ // This means we'll iterate FDMaps shared by multiple tasks repeatedly,
+ // but ktime.Timer.Pause is idempotent so this is harmless.
if fdm := t.fds; fdm != nil {
for _, desc := range fdm.files {
if tfd, ok := desc.file.FileOperations.(*timerfd.TimerOperations); ok {
@@ -774,10 +773,10 @@ func (k *Kernel) resumeTimeLocked() {
k.timekeeper.ResumeUpdates()
for t := range k.tasks.Root.tids {
if t == t.tg.leader {
- t.tg.tm.resume()
- }
- for _, it := range t.tg.timers {
- it.ResumeTimer()
+ t.tg.itimerRealTimer.Resume()
+ for _, it := range t.tg.timers {
+ it.ResumeTimer()
+ }
}
if fdm := t.fds; fdm != nil {
for _, desc := range fdm.files {
@@ -1078,22 +1077,3 @@ func (ctx supervisorContext) Value(key interface{}) interface{} {
return nil
}
}
-
-type kernelCPUClockListener struct {
- k *Kernel
-}
-
-// Notify implements ktime.TimerListener.Notify.
-func (l kernelCPUClockListener) 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 cpuClockTicker 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.
- atomic.AddUint64(&l.k.cpuClock, 1)
-}
-
-// Destroy implements ktime.TimerListener.Destroy.
-func (l kernelCPUClockListener) Destroy() {
-}
diff --git a/pkg/sentry/kernel/task_acct.go b/pkg/sentry/kernel/task_acct.go
index ce12cdb64..d2052921e 100644
--- a/pkg/sentry/kernel/task_acct.go
+++ b/pkg/sentry/kernel/task_acct.go
@@ -21,8 +21,99 @@ import (
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"
)
+// Getitimer implements getitimer(2).
+//
+// Preconditions: The caller must be running on the task goroutine.
+func (t *Task) Getitimer(id int32) (linux.ItimerVal, error) {
+ var tm ktime.Time
+ var s ktime.Setting
+ switch id {
+ case linux.ITIMER_REAL:
+ tm, s = t.tg.itimerRealTimer.Get()
+ case linux.ITIMER_VIRTUAL:
+ tm = t.tg.UserCPUClock().Now()
+ t.tg.signalHandlers.mu.Lock()
+ s, _ = t.tg.itimerVirtSetting.At(tm)
+ t.tg.signalHandlers.mu.Unlock()
+ case linux.ITIMER_PROF:
+ tm = t.tg.CPUClock().Now()
+ t.tg.signalHandlers.mu.Lock()
+ s, _ = t.tg.itimerProfSetting.At(tm)
+ t.tg.signalHandlers.mu.Unlock()
+ default:
+ return linux.ItimerVal{}, syserror.EINVAL
+ }
+ val, iv := ktime.SpecFromSetting(tm, s)
+ return linux.ItimerVal{
+ Value: linux.DurationToTimeval(val),
+ Interval: linux.DurationToTimeval(iv),
+ }, nil
+}
+
+// Setitimer implements setitimer(2).
+//
+// Preconditions: The caller must be running on the task goroutine.
+func (t *Task) Setitimer(id int32, newitv linux.ItimerVal) (linux.ItimerVal, error) {
+ var tm ktime.Time
+ var olds ktime.Setting
+ switch id {
+ case linux.ITIMER_REAL:
+ news, err := ktime.SettingFromSpec(newitv.Value.ToDuration(), newitv.Interval.ToDuration(), t.tg.itimerRealTimer.Clock())
+ if err != nil {
+ return linux.ItimerVal{}, err
+ }
+ tm, olds = t.tg.itimerRealTimer.Swap(news)
+ case linux.ITIMER_VIRTUAL:
+ c := t.tg.UserCPUClock()
+ var err error
+ t.k.cpuClockTicker.Atomically(func() {
+ tm = c.Now()
+ var news ktime.Setting
+ news, err = ktime.SettingFromSpecAt(newitv.Value.ToDuration(), newitv.Interval.ToDuration(), tm)
+ if err != nil {
+ return
+ }
+ t.tg.signalHandlers.mu.Lock()
+ olds = t.tg.itimerVirtSetting
+ t.tg.itimerVirtSetting = news
+ t.tg.updateCPUTimersEnabledLocked()
+ t.tg.signalHandlers.mu.Unlock()
+ })
+ if err != nil {
+ return linux.ItimerVal{}, err
+ }
+ case linux.ITIMER_PROF:
+ c := t.tg.CPUClock()
+ var err error
+ t.k.cpuClockTicker.Atomically(func() {
+ tm = c.Now()
+ var news ktime.Setting
+ news, err = ktime.SettingFromSpecAt(newitv.Value.ToDuration(), newitv.Interval.ToDuration(), tm)
+ if err != nil {
+ return
+ }
+ t.tg.signalHandlers.mu.Lock()
+ olds = t.tg.itimerProfSetting
+ t.tg.itimerProfSetting = news
+ t.tg.updateCPUTimersEnabledLocked()
+ t.tg.signalHandlers.mu.Unlock()
+ })
+ if err != nil {
+ return linux.ItimerVal{}, err
+ }
+ default:
+ return linux.ItimerVal{}, syserror.EINVAL
+ }
+ oldval, oldiv := ktime.SpecFromSetting(tm, olds)
+ return linux.ItimerVal{
+ Value: linux.DurationToTimeval(oldval),
+ Interval: linux.DurationToTimeval(oldiv),
+ }, nil
+}
+
// IOUsage returns the io usage of the thread.
func (t *Task) IOUsage() *usage.IO {
return t.ioUsage
@@ -56,12 +147,6 @@ func (t *Task) SetName(name string) {
t.Debugf("Set thread name to %q", name)
}
-// SetCPUTimer is used by setrlimit(RLIMIT_CPU) to enforce the hard and soft
-// limits on CPU time used by this process.
-func (tg *ThreadGroup) SetCPUTimer(l *limits.Limit) {
- tg.Timer().applyCPULimits(*l)
-}
-
// Limits implements context.Context.Limits.
func (t *Task) Limits() *limits.LimitSet {
return t.ThreadGroup().Limits()
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index 7c469ec46..de3aef40d 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -241,7 +241,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
if opts.NewSignalHandlers {
sh = sh.Fork()
}
- tg = NewThreadGroup(pidns, sh, opts.TerminationSignal, tg.limits.GetCopy(), t.k.monotonicClock)
+ tg = t.k.newThreadGroup(pidns, sh, opts.TerminationSignal, tg.limits.GetCopy(), t.k.monotonicClock)
}
cfg := &TaskConfig{
diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go
index f5b45fb17..65969ca9b 100644
--- a/pkg/sentry/kernel/task_exit.go
+++ b/pkg/sentry/kernel/task_exit.go
@@ -675,9 +675,6 @@ func (t *Task) exitNotifyLocked(fromPtraceDetach bool) {
t.tg.ioUsage.Accumulate(t.ioUsage)
t.tg.signalHandlers.mu.Lock()
t.tg.tasks.Remove(t)
- if t.tg.lastTimerSignalTask == t {
- t.tg.lastTimerSignalTask = nil
- }
t.tg.tasksCount--
tc := t.tg.tasksCount
t.tg.signalHandlers.mu.Unlock()
diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go
index 8dd0ef6ea..49ac933b7 100644
--- a/pkg/sentry/kernel/task_run.go
+++ b/pkg/sentry/kernel/task_run.go
@@ -70,10 +70,6 @@ func (t *Task) run(threadID uintptr) {
// Platform.CooperativelySharesAddressSpace() == true, we give up the
// AddressSpace before the task goroutine finishes executing.
- // Ensure that thread group timers for execution time reflect that this
- // task now exists.
- t.tg.tm.kick()
-
// If this is a newly-started task, it should check for participation in
// group stops. If this is a task resuming after restore, it was
// interrupted by saving. In either case, the task is initially
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 {
diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go
index afb010f60..e2925a708 100644
--- a/pkg/sentry/kernel/task_signals.go
+++ b/pkg/sentry/kernel/task_signals.go
@@ -359,72 +359,6 @@ func (tg *ThreadGroup) SendSignal(info *arch.SignalInfo) error {
return tg.leader.sendSignalLocked(info, true /* group */)
}
-// Preconditions: The TaskSet mutex must be locked.
-func (t *Task) onCPULocked(includeSys bool) bool {
- // Task is exiting.
- if t.exitState != TaskExitNone {
- return false
- }
-
- switch t.TaskGoroutineSchedInfo().State {
- case TaskGoroutineRunningSys:
- return includeSys
- case TaskGoroutineRunningApp:
- return true
- default:
- return false
- }
-}
-
-// SendTimerSignal mimics the process timer signal delivery behavior in linux:
-// signals are delivered to the thread that triggers the timer expiration (see
-// kernel/time/posix-cpu-timers.c:check_process_timers(). This
-// means
-// 1) the thread is running on cpu at the time.
-// 2) a thread runs more frequently will get more of those signals.
-//
-// We approximate this behavior by selecting a running task in a round-robin
-// fashion. Statistically, a thread running more often should have a higher
-// probability to be selected.
-func (tg *ThreadGroup) SendTimerSignal(info *arch.SignalInfo, includeSys bool) error {
- tg.pidns.owner.mu.RLock()
- defer tg.pidns.owner.mu.RUnlock()
- tg.signalHandlers.mu.Lock()
- defer tg.signalHandlers.mu.Unlock()
-
- // Find the next running threads.
- var t *Task
- if tg.lastTimerSignalTask == nil {
- t = tg.tasks.Front()
- } else {
- t = tg.lastTimerSignalTask.Next()
- }
-
- // Iterate from lastTimerSignalTask.Next() to the last task in the task list.
- for t != nil {
- if t.onCPULocked(includeSys) {
- tg.lastTimerSignalTask = t
- return t.sendSignalLocked(info, true /* group */)
- }
- t = t.Next()
- }
-
- // t is nil when we reach here. If lastTimerSignalTask is not nil, iterate
- // from Front to lastTimerSignalTask.
- if tg.lastTimerSignalTask != nil {
- for t := tg.tasks.Front(); t != tg.lastTimerSignalTask.Next(); t = t.Next() {
- if t.onCPULocked(includeSys) {
- tg.lastTimerSignalTask = t
- return t.sendSignalLocked(info, true /* group */)
- }
- }
- }
-
- // No running threads? Just try the leader.
- tg.lastTimerSignalTask = tg.leader
- return tg.leader.sendSignalLocked(info, true /* group */)
-}
-
func (t *Task) sendSignalLocked(info *arch.SignalInfo, group bool) error {
return t.sendSignalTimerLocked(info, group, nil)
}
diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go
index 13dce08ce..dfff7b52d 100644
--- a/pkg/sentry/kernel/thread_group.go
+++ b/pkg/sentry/kernel/thread_group.go
@@ -19,6 +19,7 @@ import (
"sync/atomic"
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time"
"gvisor.googlesource.com/gvisor/pkg/sentry/limits"
"gvisor.googlesource.com/gvisor/pkg/sentry/usage"
)
@@ -59,12 +60,6 @@ type ThreadGroup struct {
// pendingSignals is protected by the signal mutex.
pendingSignals pendingSignals
- // lastTimerSignalTask records the last task we deliver a process timer signal to.
- // Please see SendTimerSignal for more details.
- //
- // lastTimerSignalTask is protected by the signal mutex.
- lastTimerSignalTask *Task
-
// groupStopPhase indicates the state of a group stop in progress on the
// thread group, if any.
//
@@ -152,14 +147,39 @@ type ThreadGroup struct {
// restarted by Task.Start.
liveGoroutines sync.WaitGroup `state:"nosave"`
- // tm contains process timers. TimerManager fields are immutable.
- tm TimerManager
+ timerMu sync.Mutex `state:"nosave"`
+
+ // itimerRealTimer implements ITIMER_REAL for the thread group.
+ itimerRealTimer *ktime.Timer
+
+ // itimerVirtSetting is the ITIMER_VIRTUAL setting for the thread group.
+ //
+ // itimerVirtSetting is protected by the signal mutex.
+ itimerVirtSetting ktime.Setting
+
+ // itimerProfSetting is the ITIMER_PROF setting for the thread group.
+ //
+ // itimerProfSetting is protected by the signal mutex.
+ itimerProfSetting ktime.Setting
+
+ // rlimitCPUSoftSetting is the setting for RLIMIT_CPU soft limit
+ // notifications for the thread group.
+ //
+ // rlimitCPUSoftSetting is protected by the signal mutex.
+ rlimitCPUSoftSetting ktime.Setting
+
+ // cpuTimersEnabled is non-zero if itimerVirtSetting.Enabled is true,
+ // itimerProfSetting.Enabled is true, rlimitCPUSoftSetting.Enabled is true,
+ // or limits.Get(CPU) is finite.
+ //
+ // cpuTimersEnabled is protected by the signal mutex. cpuTimersEnabled is
+ // accessed using atomic memory operations.
+ cpuTimersEnabled uint32
// timers is the thread group's POSIX interval timers. nextTimerID is the
// TimerID at which allocation should begin searching for an unused ID.
//
// timers and nextTimerID are protected by timerMu.
- timerMu sync.Mutex `state:"nosave"`
timers map[linux.TimerID]*IntervalTimer
nextTimerID linux.TimerID
@@ -211,11 +231,11 @@ type ThreadGroup struct {
rscr atomic.Value `state:".(*RSEQCriticalRegion)"`
}
-// NewThreadGroup returns a new, empty thread group in PID namespace ns. The
+// newThreadGroup returns a new, empty thread group in PID namespace ns. The
// thread group leader will send its parent terminationSignal when it exits.
// The new thread group isn't visible to the system until a task has been
// created inside of it by a successful call to TaskSet.NewTask.
-func NewThreadGroup(ns *PIDNamespace, sh *SignalHandlers, terminationSignal linux.Signal, limits *limits.LimitSet, monotonicClock *timekeeperClock) *ThreadGroup {
+func (k *Kernel) newThreadGroup(ns *PIDNamespace, sh *SignalHandlers, terminationSignal linux.Signal, limits *limits.LimitSet, monotonicClock *timekeeperClock) *ThreadGroup {
tg := &ThreadGroup{
threadGroupNode: threadGroupNode{
pidns: ns,
@@ -225,7 +245,7 @@ func NewThreadGroup(ns *PIDNamespace, sh *SignalHandlers, terminationSignal linu
ioUsage: &usage.IO{},
limits: limits,
}
- tg.tm = newTimerManager(tg, monotonicClock)
+ tg.itimerRealTimer = ktime.NewTimer(k.monotonicClock, &itimerRealListener{tg: tg})
tg.timers = make(map[linux.TimerID]*IntervalTimer)
tg.rscr.Store(&RSEQCriticalRegion{})
return tg
@@ -249,11 +269,6 @@ func (tg *ThreadGroup) SignalHandlers() *SignalHandlers {
return tg.signalHandlers
}
-// Timer returns tg's timers.
-func (tg *ThreadGroup) Timer() *TimerManager {
- return &tg.tm
-}
-
// Limits returns tg's limits.
func (tg *ThreadGroup) Limits() *limits.LimitSet {
return tg.limits
@@ -261,11 +276,9 @@ func (tg *ThreadGroup) Limits() *limits.LimitSet {
// release releases the thread group's resources.
func (tg *ThreadGroup) release() {
- // These must be done without holding the TaskSet or signal mutexes since
- // timers send signals with Timer.mu locked.
-
- tg.tm.destroy()
-
+ // Timers must be destroyed without holding the TaskSet or signal mutexes
+ // since timers send signals with Timer.mu locked.
+ tg.itimerRealTimer.Destroy()
var its []*IntervalTimer
tg.pidns.owner.mu.Lock()
tg.signalHandlers.mu.Lock()
@@ -292,3 +305,19 @@ func (tg *ThreadGroup) forEachChildThreadGroupLocked(fn func(*ThreadGroup)) {
}
}
}
+
+// itimerRealListener implements ktime.Listener for ITIMER_REAL expirations.
+//
+// +stateify savable
+type itimerRealListener struct {
+ tg *ThreadGroup
+}
+
+// Notify implements ktime.TimerListener.Notify.
+func (l *itimerRealListener) Notify(exp uint64) {
+ l.tg.SendSignal(sigPriv(linux.SIGALRM))
+}
+
+// Destroy implements ktime.TimerListener.Destroy.
+func (l *itimerRealListener) Destroy() {
+}
diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go
index 3d5713106..4e3d19e97 100644
--- a/pkg/sentry/kernel/threads.go
+++ b/pkg/sentry/kernel/threads.go
@@ -243,9 +243,13 @@ func (ns *PIDNamespace) Tasks() []*Task {
// ThreadGroups returns a snapshot of the thread groups in ns.
func (ns *PIDNamespace) ThreadGroups() []*ThreadGroup {
+ return ns.ThreadGroupsAppend(nil)
+}
+
+// ThreadGroupsAppend appends a snapshot of the thread groups in ns to tgs.
+func (ns *PIDNamespace) ThreadGroupsAppend(tgs []*ThreadGroup) []*ThreadGroup {
ns.owner.mu.RLock()
defer ns.owner.mu.RUnlock()
- var tgs []*ThreadGroup
for t := range ns.tids {
if t == t.tg.leader {
tgs = append(tgs, t.tg)
diff --git a/pkg/sentry/kernel/time/time.go b/pkg/sentry/kernel/time/time.go
index 1f6fed007..52e0dfba1 100644
--- a/pkg/sentry/kernel/time/time.go
+++ b/pkg/sentry/kernel/time/time.go
@@ -307,6 +307,12 @@ type Setting struct {
// SettingFromSpec converts a (value, interval) pair to a Setting based on a
// reading from c. value is interpreted as a time relative to c.Now().
func SettingFromSpec(value time.Duration, interval time.Duration, c Clock) (Setting, error) {
+ return SettingFromSpecAt(value, interval, c.Now())
+}
+
+// SettingFromSpecAt converts a (value, interval) pair to a Setting. value is
+// interpreted as a time relative to now.
+func SettingFromSpecAt(value time.Duration, interval time.Duration, now Time) (Setting, error) {
if value < 0 {
return Setting{}, syserror.EINVAL
}
@@ -315,7 +321,7 @@ func SettingFromSpec(value time.Duration, interval time.Duration, c Clock) (Sett
}
return Setting{
Enabled: true,
- Next: c.Now().Add(value),
+ Next: now.Add(value),
Period: interval,
}, nil
}
@@ -365,14 +371,14 @@ func ItimerspecFromSetting(now Time, s Setting) linux.Itimerspec {
}
}
-// advancedTo returns an updated Setting and a number of expirations after
-// the associated Clock indicates a time of now.
+// At returns an updated Setting and a number of expirations after the
+// associated Clock indicates a time of now.
//
-// Settings may be created by successive calls to advancedTo with decreasing
+// Settings may be created by successive calls to At with decreasing
// values of now (i.e. time may appear to go backward). Supporting this is
// required to support non-monotonic clocks, as well as allowing
// Timer.clock.Now() to be called without holding Timer.mu.
-func (s Setting) advancedTo(now Time) (Setting, uint64) {
+func (s Setting) At(now Time) (Setting, uint64) {
if !s.Enabled {
return s, 0
}
@@ -519,7 +525,7 @@ func (t *Timer) Tick() {
if t.paused {
return
}
- s, exp := t.setting.advancedTo(now)
+ s, exp := t.setting.At(now)
t.setting = s
if exp > 0 {
t.listener.Notify(exp)
@@ -574,7 +580,7 @@ func (t *Timer) Get() (Time, Setting) {
if t.paused {
panic(fmt.Sprintf("Timer.Get called on paused Timer %p", t))
}
- s, exp := t.setting.advancedTo(now)
+ s, exp := t.setting.At(now)
t.setting = s
if exp > 0 {
t.listener.Notify(exp)
@@ -607,14 +613,14 @@ func (t *Timer) SwapAnd(s Setting, f func()) (Time, Setting) {
if t.paused {
panic(fmt.Sprintf("Timer.SwapAnd called on paused Timer %p", t))
}
- oldS, oldExp := t.setting.advancedTo(now)
+ oldS, oldExp := t.setting.At(now)
if oldExp > 0 {
t.listener.Notify(oldExp)
}
if f != nil {
f()
}
- newS, newExp := s.advancedTo(now)
+ newS, newExp := s.At(now)
t.setting = newS
if newExp > 0 {
t.listener.Notify(newExp)
@@ -623,6 +629,17 @@ func (t *Timer) SwapAnd(s Setting, f func()) (Time, Setting) {
return now, oldS
}
+// Atomically invokes f atomically with respect to expirations of t; that is, t
+// cannot generate expirations while f is being called.
+//
+// Preconditions: f cannot call any Timer methods since it is called with the
+// Timer mutex locked.
+func (t *Timer) Atomically(f func()) {
+ t.mu.Lock()
+ defer t.mu.Unlock()
+ f()
+}
+
// Preconditions: t.mu must be locked.
func (t *Timer) resetKickerLocked(now Time) {
if t.setting.Enabled {
diff --git a/pkg/sentry/kernel/timekeeper.go b/pkg/sentry/kernel/timekeeper.go
index df5dbe128..2167f3efe 100644
--- a/pkg/sentry/kernel/timekeeper.go
+++ b/pkg/sentry/kernel/timekeeper.go
@@ -15,6 +15,7 @@
package kernel
import (
+ "fmt"
"sync"
"time"
@@ -277,3 +278,28 @@ func (t *Timekeeper) GetTime(c sentrytime.ClockID) (int64, error) {
func (t *Timekeeper) BootTime() ktime.Time {
return t.bootTime
}
+
+// timekeeperClock is a ktime.Clock that reads time from a
+// kernel.Timekeeper-managed clock.
+//
+// +stateify savable
+type timekeeperClock struct {
+ tk *Timekeeper
+ c sentrytime.ClockID
+
+ // Implements ktime.Clock.WallTimeUntil.
+ ktime.WallRateClock `state:"nosave"`
+
+ // Implements waiter.Waitable. (We have no ability to detect
+ // discontinuities from external changes to CLOCK_REALTIME).
+ ktime.NoClockEvents `state:"nosave"`
+}
+
+// Now implements ktime.Clock.Now.
+func (tc *timekeeperClock) Now() ktime.Time {
+ now, err := tc.tk.GetTime(tc.c)
+ if err != nil {
+ panic(fmt.Sprintf("timekeeperClock(ClockID=%v)).Now: %v", tc.c, err))
+ }
+ return ktime.FromNanoseconds(now)
+}
diff --git a/pkg/sentry/kernel/timer.go b/pkg/sentry/kernel/timer.go
deleted file mode 100644
index 534d03d0f..000000000
--- a/pkg/sentry/kernel/timer.go
+++ /dev/null
@@ -1,290 +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 (
- "fmt"
- "time"
-
- "gvisor.googlesource.com/gvisor/pkg/abi/linux"
- ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time"
- "gvisor.googlesource.com/gvisor/pkg/sentry/limits"
- sentrytime "gvisor.googlesource.com/gvisor/pkg/sentry/time"
-)
-
-// timekeeperClock is a ktime.Clock that reads time from a
-// kernel.Timekeeper-managed clock.
-//
-// +stateify savable
-type timekeeperClock struct {
- tk *Timekeeper
- c sentrytime.ClockID
-
- // Implements ktime.Clock.WallTimeUntil.
- ktime.WallRateClock `state:"nosave"`
-
- // Implements waiter.Waitable. (We have no ability to detect
- // discontinuities from external changes to CLOCK_REALTIME).
- ktime.NoClockEvents `state:"nosave"`
-}
-
-// Now implements ktime.Clock.Now.
-func (tc *timekeeperClock) Now() ktime.Time {
- now, err := tc.tk.GetTime(tc.c)
- if err != nil {
- panic(fmt.Sprintf("timekeeperClock(ClockID=%v)).Now: %v", tc.c, err))
- }
- return ktime.FromNanoseconds(now)
-}
-
-// tgClock is a ktime.Clock that measures the time a thread group has spent
-// executing.
-//
-// +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"`
-}
-
-// UserCPUClock returns a ktime.Clock that measures the time that a thread
-// group has spent executing.
-func (tg *ThreadGroup) UserCPUClock() ktime.Clock {
- return tg.tm.virtClock
-}
-
-// 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 tg.tm.profClock
-}
-
-// 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 {
- // The assumption here is that the time spent in this process (not matter
- // virtual or prof) should not exceed wall time * active tasks, since
- // Task.exitThreadGroup stops accounting as it transitions to
- // TaskExitInitiated.
- tgc.tg.pidns.owner.mu.RLock()
- n := tgc.tg.activeTasks
- 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(int64(t.Sub(now))/int64(n)) * time.Nanosecond
- return ((remaining + (linux.ClockTick - time.Nanosecond)) / linux.ClockTick) * linux.ClockTick
-}
-
-// taskClock is a ktime.Clock that measures the time that a task has spent
-// executing.
-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())
-}
-
-// signalNotifier is a ktime.Listener that sends signals to a ThreadGroup.
-//
-// +stateify savable
-type signalNotifier struct {
- tg *ThreadGroup
- signal linux.Signal
- realTimer bool
- includeSys bool
-}
-
-// Notify implements ktime.TimerListener.Notify.
-func (s *signalNotifier) Notify(exp uint64) {
- // Since all signals sent using a signalNotifier are standard (not
- // real-time) signals, we can ignore the number of expirations and send
- // only a single signal.
- if s.realTimer {
- // real timer signal sent to leader. See kernel/time/itimer.c:it_real_fn
- s.tg.SendSignal(sigPriv(s.signal))
- } else {
- s.tg.SendTimerSignal(sigPriv(s.signal), s.includeSys)
- }
-}
-
-// Destroy implements ktime.TimerListener.Destroy.
-func (s *signalNotifier) Destroy() {}
-
-// TimerManager is a collection of supported process cpu timers.
-//
-// +stateify savable
-type TimerManager struct {
- // Clocks used to drive thread group execution time timers.
- virtClock *tgClock
- profClock *tgClock
-
- RealTimer *ktime.Timer
- VirtualTimer *ktime.Timer
- ProfTimer *ktime.Timer
- SoftLimitTimer *ktime.Timer
- HardLimitTimer *ktime.Timer
-}
-
-// newTimerManager returns a new instance of TimerManager.
-func newTimerManager(tg *ThreadGroup, monotonicClock ktime.Clock) TimerManager {
- virtClock := &tgClock{tg: tg, includeSys: false}
- profClock := &tgClock{tg: tg, includeSys: true}
- tm := TimerManager{
- virtClock: virtClock,
- profClock: profClock,
- RealTimer: ktime.NewTimer(monotonicClock, &signalNotifier{
- tg: tg,
- signal: linux.SIGALRM,
- realTimer: true,
- includeSys: false,
- }),
- VirtualTimer: ktime.NewTimer(virtClock, &signalNotifier{
- tg: tg,
- signal: linux.SIGVTALRM,
- realTimer: false,
- includeSys: false,
- }),
- ProfTimer: ktime.NewTimer(profClock, &signalNotifier{
- tg: tg,
- signal: linux.SIGPROF,
- realTimer: false,
- includeSys: true,
- }),
- SoftLimitTimer: ktime.NewTimer(profClock, &signalNotifier{
- tg: tg,
- signal: linux.SIGXCPU,
- realTimer: false,
- includeSys: true,
- }),
- HardLimitTimer: ktime.NewTimer(profClock, &signalNotifier{
- tg: tg,
- signal: linux.SIGKILL,
- realTimer: false,
- includeSys: true,
- }),
- }
- tm.applyCPULimits(tg.Limits().Get(limits.CPU))
- return tm
-}
-
-// Save saves this TimerManger.
-
-// destroy destroys all timers.
-func (tm *TimerManager) destroy() {
- tm.RealTimer.Destroy()
- tm.VirtualTimer.Destroy()
- tm.ProfTimer.Destroy()
- tm.SoftLimitTimer.Destroy()
- tm.HardLimitTimer.Destroy()
-}
-
-func (tm *TimerManager) applyCPULimits(l limits.Limit) {
- tm.SoftLimitTimer.Swap(ktime.Setting{
- Enabled: l.Cur != limits.Infinity,
- Next: ktime.FromNanoseconds((time.Duration(l.Cur) * time.Second).Nanoseconds()),
- Period: time.Second,
- })
- tm.HardLimitTimer.Swap(ktime.Setting{
- Enabled: l.Max != limits.Infinity,
- Next: ktime.FromNanoseconds((time.Duration(l.Max) * time.Second).Nanoseconds()),
- })
-}
-
-// kick is called when the number of threads in the thread group associated
-// with tm increases.
-func (tm *TimerManager) kick() {
- tm.virtClock.Notify(ktime.ClockEventRateIncrease)
- tm.profClock.Notify(ktime.ClockEventRateIncrease)
-}
-
-// pause is to pause the timers and stop timer signal delivery.
-func (tm *TimerManager) pause() {
- tm.RealTimer.Pause()
- tm.VirtualTimer.Pause()
- tm.ProfTimer.Pause()
- tm.SoftLimitTimer.Pause()
- tm.HardLimitTimer.Pause()
-}
-
-// resume is to resume the timers and continue timer signal delivery.
-func (tm *TimerManager) resume() {
- tm.RealTimer.Resume()
- tm.VirtualTimer.Resume()
- tm.ProfTimer.Resume()
- tm.SoftLimitTimer.Resume()
- tm.HardLimitTimer.Resume()
-}
diff --git a/pkg/sentry/syscalls/linux/sys_rlimit.go b/pkg/sentry/syscalls/linux/sys_rlimit.go
index 481e79eaa..d806b58ab 100644
--- a/pkg/sentry/syscalls/linux/sys_rlimit.go
+++ b/pkg/sentry/syscalls/linux/sys_rlimit.go
@@ -111,7 +111,7 @@ func prlimit64(t *kernel.Task, resource limits.LimitType, newLim *limits.Limit)
}
if resource == limits.CPU {
- t.ThreadGroup().SetCPUTimer(newLim)
+ t.NotifyRlimitCPUUpdated()
}
return oldLim, nil
}
diff --git a/pkg/sentry/syscalls/linux/sys_timer.go b/pkg/sentry/syscalls/linux/sys_timer.go
index a12d12d9d..c41074d54 100644
--- a/pkg/sentry/syscalls/linux/sys_timer.go
+++ b/pkg/sentry/syscalls/linux/sys_timer.go
@@ -21,7 +21,6 @@ import (
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
"gvisor.googlesource.com/gvisor/pkg/sentry/arch"
"gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
- ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time"
"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
)
@@ -70,34 +69,15 @@ func copyItimerValOut(t *kernel.Task, addr usermem.Addr, itv *linux.ItimerVal) e
}
}
-func findTimer(t *kernel.Task, which int32) (*ktime.Timer, error) {
- switch which {
- case linux.ITIMER_REAL:
- return t.ThreadGroup().Timer().RealTimer, nil
- case linux.ITIMER_VIRTUAL:
- return t.ThreadGroup().Timer().VirtualTimer, nil
- case linux.ITIMER_PROF:
- return t.ThreadGroup().Timer().ProfTimer, nil
- default:
- return nil, syscall.EINVAL
- }
-}
-
// Getitimer implements linux syscall getitimer(2).
func Getitimer(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
timerID := args[0].Int()
val := args[1].Pointer()
- timer, err := findTimer(t, timerID)
+ olditv, err := t.Getitimer(timerID)
if err != nil {
return 0, nil, err
}
- value, interval := ktime.SpecFromSetting(timer.Get())
- olditv := linux.ItimerVal{
- Value: linux.DurationToTimeval(value),
- Interval: linux.DurationToTimeval(interval),
- }
-
return 0, nil, copyItimerValOut(t, val, &olditv)
}
@@ -107,29 +87,14 @@ func Setitimer(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sys
newVal := args[1].Pointer()
oldVal := args[2].Pointer()
- timer, err := findTimer(t, timerID)
+ newitv, err := copyItimerValIn(t, newVal)
if err != nil {
return 0, nil, err
}
-
- itv, err := copyItimerValIn(t, newVal)
+ olditv, err := t.Setitimer(timerID, newitv)
if err != nil {
return 0, nil, err
}
- // Just like linux, we cap the timer value and interval with the max
- // number that int64 can represent which is roughly 292 years.
- s, err := ktime.SettingFromSpec(itv.Value.ToDuration(),
- itv.Interval.ToDuration(), timer.Clock())
- if err != nil {
- return 0, nil, err
- }
-
- valueNS, intervalNS := ktime.SpecFromSetting(timer.Swap(s))
- olditv := linux.ItimerVal{
- Value: linux.DurationToTimeval(valueNS),
- Interval: linux.DurationToTimeval(intervalNS),
- }
-
return 0, nil, copyItimerValOut(t, oldVal, &olditv)
}
@@ -137,21 +102,19 @@ func Setitimer(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sys
func Alarm(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
duration := time.Duration(args[0].Uint()) * time.Second
- timer := t.ThreadGroup().Timer().RealTimer
- s, err := ktime.SettingFromSpec(duration, 0, timer.Clock())
+ olditv, err := t.Setitimer(linux.ITIMER_REAL, linux.ItimerVal{
+ Value: linux.DurationToTimeval(duration),
+ })
if err != nil {
return 0, nil, err
}
-
- value, _ := ktime.SpecFromSetting(timer.Swap(s))
- sec := int64(value) / nsecPerSec
- nsec := int64(value) % nsecPerSec
- // We can't return 0 if we have an alarm pending ...
- if (sec == 0 && nsec > 0) || nsec >= nsecPerSec/2 {
- sec++
+ olddur := olditv.Value.ToDuration()
+ secs := olddur.Round(time.Second).Nanoseconds() / nsecPerSec
+ if secs == 0 && olddur != 0 {
+ // We can't return 0 if an alarm was previously scheduled.
+ secs = 1
}
-
- return uintptr(sec), nil, nil
+ return uintptr(secs), nil, nil
}
// TimerCreate implements linux syscall timer_create(2).