summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry
diff options
context:
space:
mode:
authorMichael Pratt <mpratt@google.com>2019-10-01 12:13:09 -0700
committergVisor bot <gvisor-bot@google.com>2019-10-01 12:21:01 -0700
commitdd69b49ed1103bab82a6b2ac95221b89b46f3376 (patch)
tree35671dd04b4672b892f1c36d53d51ed6a81c8a1a /pkg/sentry
parent90e908f4198d718236758625c54af89201c8c1e7 (diff)
Disable cpuClockTicker when app is idle
Kernel.cpuClockTicker increments kernel.cpuClock, which tasks use as a clock to track their CPU usage. This improves latency in the syscall path by avoid expensive monotonic clock calls on every syscall entry/exit. However, this timer fires every 10ms. Thus, when all tasks are idle (i.e., blocked or stopped), this forces a sentry wakeup every 10ms, when we may otherwise be able to sleep until the next app-relevant event. These wakeups cause the sentry to utilize approximately 2% CPU when the application is otherwise idle. Updates to clock are not strictly necessary when the app is idle, as there are no readers of cpuClock. This commit reduces idle CPU by disabling the timer when tasks are completely idle, and computing its effects at the next wakeup. Rather than disabling the timer as soon as the app goes idle, we wait until the next tick, which provides a window for short sleeps to sleep and wakeup without doing the (relatively) expensive work of disabling and enabling the timer. PiperOrigin-RevId: 272265822
Diffstat (limited to 'pkg/sentry')
-rw-r--r--pkg/sentry/fs/timerfd/timerfd.go3
-rw-r--r--pkg/sentry/kernel/kernel.go129
-rw-r--r--pkg/sentry/kernel/posixtimer.go8
-rw-r--r--pkg/sentry/kernel/task_sched.go33
-rw-r--r--pkg/sentry/kernel/thread_group.go3
-rw-r--r--pkg/sentry/kernel/time/time.go27
6 files changed, 190 insertions, 13 deletions
diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go
index 59403d9db..f8bf663bb 100644
--- a/pkg/sentry/fs/timerfd/timerfd.go
+++ b/pkg/sentry/fs/timerfd/timerfd.go
@@ -141,9 +141,10 @@ func (t *TimerOperations) Write(context.Context, *fs.File, usermem.IOSequence, i
}
// Notify implements ktime.TimerListener.Notify.
-func (t *TimerOperations) Notify(exp uint64) {
+func (t *TimerOperations) Notify(exp uint64, setting ktime.Setting) (ktime.Setting, bool) {
atomic.AddUint64(&t.val, exp)
t.events.Notify(waiter.EventIn)
+ return ktime.Setting{}, false
}
// Destroy implements ktime.TimerListener.Destroy.
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index 8c1f79ab5..3cda03891 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -24,6 +24,7 @@
// TaskSet.mu
// SignalHandlers.mu
// Task.mu
+// runningTasksMu
//
// Locking SignalHandlers.mu in multiple SignalHandlers requires locking
// TaskSet.mu exclusively first. Locking Task.mu in multiple Tasks at the same
@@ -135,6 +136,22 @@ type Kernel struct {
// syslog is the kernel log.
syslog syslog
+ // runningTasksMu synchronizes disable/enable of cpuClockTicker when
+ // the kernel is idle (runningTasks == 0).
+ //
+ // runningTasksMu is used to exclude critical sections when the timer
+ // disables itself and when the first active task enables the timer,
+ // ensuring that tasks always see a valid cpuClock value.
+ runningTasksMu sync.Mutex `state:"nosave"`
+
+ // runningTasks is the total count of tasks currently in
+ // TaskGoroutineRunningSys or TaskGoroutineRunningApp. i.e., they are
+ // not blocked or stopped.
+ //
+ // runningTasks must be accessed atomically. Increments from 0 to 1 are
+ // further protected by runningTasksMu (see incRunningTasks).
+ runningTasks int64
+
// cpuClock is incremented every linux.ClockTick. cpuClock is used to
// measure task CPU usage, since sampling monotonicClock twice on every
// syscall turns out to be unreasonably expensive. This is similar to how
@@ -150,6 +167,22 @@ type Kernel struct {
// cpuClockTicker increments cpuClock.
cpuClockTicker *ktime.Timer `state:"nosave"`
+ // cpuClockTickerDisabled indicates that cpuClockTicker has been
+ // disabled because no tasks are running.
+ //
+ // cpuClockTickerDisabled is protected by runningTasksMu.
+ cpuClockTickerDisabled bool
+
+ // cpuClockTickerSetting is the ktime.Setting of cpuClockTicker at the
+ // point it was disabled. It is cached here to avoid a lock ordering
+ // violation with cpuClockTicker.mu when runningTaskMu is held.
+ //
+ // cpuClockTickerSetting is only valid when cpuClockTickerDisabled is
+ // true.
+ //
+ // cpuClockTickerSetting is protected by runningTasksMu.
+ cpuClockTickerSetting ktime.Setting
+
// fdMapUids is an ever-increasing counter for generating FDTable uids.
//
// fdMapUids is mutable, and is accessed using atomic memory operations.
@@ -912,6 +945,102 @@ func (k *Kernel) resumeTimeLocked() {
}
}
+func (k *Kernel) incRunningTasks() {
+ for {
+ tasks := atomic.LoadInt64(&k.runningTasks)
+ if tasks != 0 {
+ // Standard case. Simply increment.
+ if !atomic.CompareAndSwapInt64(&k.runningTasks, tasks, tasks+1) {
+ continue
+ }
+ return
+ }
+
+ // Transition from 0 -> 1. Synchronize with other transitions and timer.
+ k.runningTasksMu.Lock()
+ tasks = atomic.LoadInt64(&k.runningTasks)
+ if tasks != 0 {
+ // We're no longer the first task, no need to
+ // re-enable.
+ atomic.AddInt64(&k.runningTasks, 1)
+ k.runningTasksMu.Unlock()
+ return
+ }
+
+ if !k.cpuClockTickerDisabled {
+ // Timer was never disabled.
+ atomic.StoreInt64(&k.runningTasks, 1)
+ k.runningTasksMu.Unlock()
+ return
+ }
+
+ // We need to update cpuClock for all of the ticks missed while we
+ // slept, and then re-enable the timer.
+ //
+ // The Notify in Swap isn't sufficient. kernelCPUClockTicker.Notify
+ // always increments cpuClock by 1 regardless of the number of
+ // expirations as a heuristic to avoid over-accounting in cases of CPU
+ // throttling.
+ //
+ // We want to cover the normal case, when all time should be accounted,
+ // so we increment for all expirations. Throttling is less concerning
+ // here because the ticker is only disabled from Notify. This means
+ // that Notify must schedule and compensate for the throttled period
+ // before the timer is disabled. Throttling while the timer is disabled
+ // doesn't matter, as nothing is running or reading cpuClock anyways.
+ //
+ // S/R also adds complication, as there are two cases. Recall that
+ // monotonicClock will jump forward on restore.
+ //
+ // 1. If the ticker is enabled during save, then on Restore Notify is
+ // called with many expirations, covering the time jump, but cpuClock
+ // is only incremented by 1.
+ //
+ // 2. If the ticker is disabled during save, then after Restore the
+ // first wakeup will call this function and cpuClock will be
+ // incremented by the number of expirations across the S/R.
+ //
+ // These cause very different value of cpuClock. But again, since
+ // nothing was running while the ticker was disabled, those differences
+ // don't matter.
+ setting, exp := k.cpuClockTickerSetting.At(k.monotonicClock.Now())
+ if exp > 0 {
+ atomic.AddUint64(&k.cpuClock, exp)
+ }
+
+ // Now that cpuClock is updated it is safe to allow other tasks to
+ // transition to running.
+ atomic.StoreInt64(&k.runningTasks, 1)
+
+ // N.B. we must unlock before calling Swap to maintain lock ordering.
+ //
+ // cpuClockTickerDisabled need not wait until after Swap to become
+ // true. It is sufficient that the timer *will* be enabled.
+ k.cpuClockTickerDisabled = false
+ k.runningTasksMu.Unlock()
+
+ // This won't call Notify (unless it's been ClockTick since setting.At
+ // above). This means we skip the thread group work in Notify. However,
+ // since nothing was running while we were disabled, none of the timers
+ // could have expired.
+ k.cpuClockTicker.Swap(setting)
+
+ return
+ }
+}
+
+func (k *Kernel) decRunningTasks() {
+ tasks := atomic.AddInt64(&k.runningTasks, -1)
+ if tasks < 0 {
+ panic(fmt.Sprintf("Invalid running count %d", tasks))
+ }
+
+ // Nothing to do. The next CPU clock tick will disable the timer if
+ // there is still nothing running. This provides approximately one tick
+ // of slack in which we can switch back and forth between idle and
+ // active without an expensive transition.
+}
+
// WaitExited blocks until all tasks in k have exited.
func (k *Kernel) WaitExited() {
k.tasks.liveGoroutines.Wait()
diff --git a/pkg/sentry/kernel/posixtimer.go b/pkg/sentry/kernel/posixtimer.go
index c5d095af7..2e861a5a8 100644
--- a/pkg/sentry/kernel/posixtimer.go
+++ b/pkg/sentry/kernel/posixtimer.go
@@ -117,9 +117,9 @@ func (it *IntervalTimer) signalRejectedLocked() {
}
// Notify implements ktime.TimerListener.Notify.
-func (it *IntervalTimer) Notify(exp uint64) {
+func (it *IntervalTimer) Notify(exp uint64, setting ktime.Setting) (ktime.Setting, bool) {
if it.target == nil {
- return
+ return ktime.Setting{}, false
}
it.target.tg.pidns.owner.mu.RLock()
@@ -129,7 +129,7 @@ func (it *IntervalTimer) Notify(exp uint64) {
if it.sigpending {
it.overrunCur += exp
- return
+ return ktime.Setting{}, false
}
// sigpending must be set before sendSignalTimerLocked() so that it can be
@@ -148,6 +148,8 @@ func (it *IntervalTimer) Notify(exp uint64) {
if err := it.target.sendSignalTimerLocked(si, it.group, it); err != nil {
it.signalRejectedLocked()
}
+
+ return ktime.Setting{}, false
}
// Destroy implements ktime.TimerListener.Destroy. Users of Timer should call
diff --git a/pkg/sentry/kernel/task_sched.go b/pkg/sentry/kernel/task_sched.go
index e76c069b0..8b148db35 100644
--- a/pkg/sentry/kernel/task_sched.go
+++ b/pkg/sentry/kernel/task_sched.go
@@ -126,12 +126,22 @@ func (t *Task) accountTaskGoroutineEnter(state TaskGoroutineState) {
t.gosched.Timestamp = now
t.gosched.State = state
t.goschedSeq.EndWrite()
+
+ if state != TaskGoroutineRunningApp {
+ // Task is blocking/stopping.
+ t.k.decRunningTasks()
+ }
}
// Preconditions: The caller must be running on the task goroutine, and leaving
// a state indicated by a previous call to
// t.accountTaskGoroutineEnter(state).
func (t *Task) accountTaskGoroutineLeave(state TaskGoroutineState) {
+ if state != TaskGoroutineRunningApp {
+ // Task is unblocking/continuing.
+ t.k.incRunningTasks()
+ }
+
now := t.k.CPUClockNow()
if t.gosched.State != state {
panic(fmt.Sprintf("Task goroutine switching from state %v (expected %v) to %v", t.gosched.State, state, TaskGoroutineRunningSys))
@@ -330,7 +340,7 @@ func newKernelCPUClockTicker(k *Kernel) *kernelCPUClockTicker {
}
// Notify implements ktime.TimerListener.Notify.
-func (ticker *kernelCPUClockTicker) Notify(exp uint64) {
+func (ticker *kernelCPUClockTicker) Notify(exp uint64, setting ktime.Setting) (ktime.Setting, bool) {
// 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
@@ -426,6 +436,27 @@ func (ticker *kernelCPUClockTicker) Notify(exp uint64) {
tgs[i] = nil
}
ticker.tgs = tgs[:0]
+
+ // If nothing is running, we can disable the timer.
+ tasks := atomic.LoadInt64(&ticker.k.runningTasks)
+ if tasks == 0 {
+ ticker.k.runningTasksMu.Lock()
+ defer ticker.k.runningTasksMu.Unlock()
+ tasks := atomic.LoadInt64(&ticker.k.runningTasks)
+ if tasks != 0 {
+ // Raced with a 0 -> 1 transition.
+ return setting, false
+ }
+
+ // Stop the timer. We must cache the current setting so the
+ // kernel can access it without violating the lock order.
+ ticker.k.cpuClockTickerSetting = setting
+ ticker.k.cpuClockTickerDisabled = true
+ setting.Enabled = false
+ return setting, true
+ }
+
+ return setting, false
}
// Destroy implements ktime.TimerListener.Destroy.
diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go
index 0eef24bfb..72568d296 100644
--- a/pkg/sentry/kernel/thread_group.go
+++ b/pkg/sentry/kernel/thread_group.go
@@ -511,8 +511,9 @@ type itimerRealListener struct {
}
// Notify implements ktime.TimerListener.Notify.
-func (l *itimerRealListener) Notify(exp uint64) {
+func (l *itimerRealListener) Notify(exp uint64, setting ktime.Setting) (ktime.Setting, bool) {
l.tg.SendSignal(SignalInfoPriv(linux.SIGALRM))
+ return ktime.Setting{}, false
}
// Destroy implements ktime.TimerListener.Destroy.
diff --git a/pkg/sentry/kernel/time/time.go b/pkg/sentry/kernel/time/time.go
index aa6c75d25..107394183 100644
--- a/pkg/sentry/kernel/time/time.go
+++ b/pkg/sentry/kernel/time/time.go
@@ -280,13 +280,16 @@ func (ClockEventsQueue) Readiness(mask waiter.EventMask) waiter.EventMask {
// A TimerListener receives expirations from a Timer.
type TimerListener interface {
// Notify is called when its associated Timer expires. exp is the number of
- // expirations.
+ // expirations. setting is the next timer Setting.
//
// Notify is called with the associated Timer's mutex locked, so Notify
// must not take any locks that precede Timer.mu in lock order.
//
+ // If Notify returns true, the timer will use the returned setting
+ // rather than the passed one.
+ //
// Preconditions: exp > 0.
- Notify(exp uint64)
+ Notify(exp uint64, setting Setting) (newSetting Setting, update bool)
// Destroy is called when the timer is destroyed.
Destroy()
@@ -533,7 +536,9 @@ func (t *Timer) Tick() {
s, exp := t.setting.At(now)
t.setting = s
if exp > 0 {
- t.listener.Notify(exp)
+ if newS, ok := t.listener.Notify(exp, t.setting); ok {
+ t.setting = newS
+ }
}
t.resetKickerLocked(now)
}
@@ -588,7 +593,9 @@ func (t *Timer) Get() (Time, Setting) {
s, exp := t.setting.At(now)
t.setting = s
if exp > 0 {
- t.listener.Notify(exp)
+ if newS, ok := t.listener.Notify(exp, t.setting); ok {
+ t.setting = newS
+ }
}
t.resetKickerLocked(now)
return now, s
@@ -620,7 +627,9 @@ func (t *Timer) SwapAnd(s Setting, f func()) (Time, Setting) {
}
oldS, oldExp := t.setting.At(now)
if oldExp > 0 {
- t.listener.Notify(oldExp)
+ t.listener.Notify(oldExp, oldS)
+ // N.B. The returned Setting doesn't matter because we're about
+ // to overwrite.
}
if f != nil {
f()
@@ -628,7 +637,9 @@ func (t *Timer) SwapAnd(s Setting, f func()) (Time, Setting) {
newS, newExp := s.At(now)
t.setting = newS
if newExp > 0 {
- t.listener.Notify(newExp)
+ if newS, ok := t.listener.Notify(newExp, t.setting); ok {
+ t.setting = newS
+ }
}
t.resetKickerLocked(now)
return now, oldS
@@ -683,11 +694,13 @@ func NewChannelNotifier() (TimerListener, <-chan struct{}) {
}
// Notify implements ktime.TimerListener.Notify.
-func (c *ChannelNotifier) Notify(uint64) {
+func (c *ChannelNotifier) Notify(uint64, Setting) (Setting, bool) {
select {
case c.tchan <- struct{}{}:
default:
}
+
+ return Setting{}, false
}
// Destroy implements ktime.TimerListener.Destroy and will close the channel.