// forming a singly-linked list with the mutex's key field pointing to the head
// of the list.
type mWaitList struct {
- next muintptr // next m waiting for lock
+ next muintptr // next m waiting for lock
+ startTicks int64 // when this m started waiting for the current lock holder, in cputicks
}
// lockVerifyMSize confirms that we can recreate the low bits of the M pointer.
}
semacreate(gp.m)
- timer := &lockTimer{lock: l}
- timer.begin()
+ var startTime int64
// On uniprocessors, no point spinning.
// On multiprocessors, spin for mutexActiveSpinCount attempts.
spin := 0
spin = mutexActiveSpinCount
}
- var weSpin, atTail bool
+ var weSpin, atTail, haveTimers bool
v := atomic.Loaduintptr(&l.key)
tryAcquire:
for i := 0; ; i++ {
next = next &^ mutexSleeping
}
if atomic.Casuintptr(&l.key, v, next) {
- timer.end()
+ gp.m.mLockProfile.end(startTime)
return
}
} else {
prev8 := atomic.Xchg8(k8, mutexLocked|mutexSleeping)
if prev8&mutexLocked == 0 {
- timer.end()
+ gp.m.mLockProfile.end(startTime)
return
}
}
throw("runtimeĀ·lock: sleeping while lock is available")
}
+ // Collect times for mutex profile (seen in unlock2 only via mWaitList),
+ // and for "/sync/mutex/wait/total:seconds" metric (to match).
+ if !haveTimers {
+ gp.m.mWaitList.startTicks = cputicks()
+ startTime = gp.m.mLockProfile.start()
+ haveTimers = true
+ }
// Store the current head of the list of sleeping Ms in our gp.m.mWaitList.next field
gp.m.mWaitList.next = mutexWaitListHead(v)
func unlock2(l *mutex) {
gp := getg()
- prev8 := atomic.Xchg8(key8(&l.key), 0)
+ var prev8 uint8
+ var haveStackLock bool
+ var endTicks int64
+ if !mutexSampleContention() {
+ // Not collecting a sample for the contention profile, do the quick release
+ prev8 = atomic.Xchg8(key8(&l.key), 0)
+ } else {
+ // If there's contention, we'll sample it. Don't allow another
+ // lock2/unlock2 pair to finish before us and take our blame. Prevent
+ // that by trading for the stack lock with a CAS.
+ v := atomic.Loaduintptr(&l.key)
+ for {
+ if v&^mutexMMask == 0 || v&mutexStackLocked != 0 {
+ // No contention, or (stack lock unavailable) no way to calculate it
+ prev8 = atomic.Xchg8(key8(&l.key), 0)
+ endTicks = 0
+ break
+ }
+
+ // There's contention, the stack lock appeared to be available, and
+ // we'd like to collect a sample for the contention profile.
+ if endTicks == 0 {
+ // Read the time before releasing the lock. The profile will be
+ // strictly smaller than what other threads would see by timing
+ // their lock calls.
+ endTicks = cputicks()
+ }
+ next := (v | mutexStackLocked) &^ (mutexLocked | mutexSleeping)
+ if atomic.Casuintptr(&l.key, v, next) {
+ haveStackLock = true
+ prev8 = uint8(v)
+ // The fast path of lock2 may have cleared mutexSleeping.
+ // Restore it so we're sure to call unlock2Wake below.
+ prev8 |= mutexSleeping
+ break
+ }
+ v = atomic.Loaduintptr(&l.key)
+ }
+ }
if prev8&mutexLocked == 0 {
throw("unlock of unlocked lock")
}
if prev8&mutexSleeping != 0 {
- unlock2Wake(l)
+ unlock2Wake(l, haveStackLock, endTicks)
}
- gp.m.mLockProfile.recordUnlock(l)
+ gp.m.mLockProfile.store()
gp.m.locks--
if gp.m.locks < 0 {
throw("runtimeĀ·unlock: lock count")
}
}
+// mutexSampleContention returns whether the current mutex operation should
+// report any contention it discovers.
+func mutexSampleContention() bool {
+ if rate := int64(atomic.Load64(&mutexprofilerate)); rate <= 0 {
+ return false
+ } else {
+ // TODO: have SetMutexProfileFraction do the clamping
+ rate32 := uint32(rate)
+ if int64(rate32) != rate {
+ rate32 = ^uint32(0)
+ }
+ return cheaprandn(rate32) == 0
+ }
+}
+
// unlock2Wake updates the list of Ms waiting on l, waking an M if necessary.
//
//go:nowritebarrier
-func unlock2Wake(l *mutex) {
+func unlock2Wake(l *mutex, haveStackLock bool, endTicks int64) {
v := atomic.Loaduintptr(&l.key)
// On occasion, seek out and wake the M at the bottom of the stack so it
// doesn't starve.
antiStarve := cheaprandn(mutexTailWakePeriod) == 0
+
+ if haveStackLock {
+ goto useStackLock
+ }
+
if !(antiStarve || // avoiding starvation may require a wake
v&mutexSpinning == 0 || // no spinners means we must wake
mutexPreferLowLatency(l)) { // prefer waiters be awake as much as possible
// We own the mutexStackLocked flag. New Ms may push themselves onto the
// stack concurrently, but we're now the only thread that can remove or
// modify the Ms that are sleeping in the list.
+useStackLock:
+
+ if endTicks != 0 {
+ // Find the M at the bottom of the stack of waiters, which has been
+ // asleep for the longest. Take the average of its wait time and the
+ // head M's wait time for the mutex contention profile, matching the
+ // estimate we do in semrelease1 (for sync.Mutex contention).
+ //
+ // We don't keep track of the tail node (we don't need it often), so do
+ // an O(N) walk on the list of sleeping Ms to find it.
+ head := mutexWaitListHead(v).ptr()
+ for node, n := head, 0; ; {
+ n++
+ next := node.mWaitList.next.ptr()
+ if next == nil {
+ cycles := endTicks - (head.mWaitList.startTicks+node.mWaitList.startTicks)/2
+ node.mWaitList.startTicks = endTicks
+ head.mWaitList.startTicks = endTicks
+ getg().m.mLockProfile.recordUnlock(cycles * int64(n))
+ break
+ }
+ node = next
+ }
+ }
var committed *m // If we choose an M within the stack, we've made a promise to wake it
for {
prev, wakem = wakem, next
}
if wakem != mp {
- prev.mWaitList.next = wakem.mWaitList.next
committed = wakem
+ prev.mWaitList.next = wakem.mWaitList.next
+ // An M sets its own startTicks when it first goes to sleep.
+ // When an unlock operation is sampled for the mutex
+ // contention profile, it takes blame for the entire list of
+ // waiting Ms but only updates the startTicks value at the
+ // tail. Copy any updates to the next-oldest M.
+ prev.mWaitList.startTicks = wakem.mWaitList.startTicks
}
}
}
// Claimed an M. Wake it.
semawakeup(wakem)
}
- break
+ return
}
v = atomic.Loaduintptr(&l.key)
return n
}
-// lockTimer assists with profiling contention on runtime-internal locks.
+// mLockProfile holds information about the runtime-internal lock contention
+// experienced and caused by this M, to report in metrics and profiles.
//
-// There are several steps between the time that an M experiences contention and
-// when that contention may be added to the profile. This comes from our
-// constraints: We need to keep the critical section of each lock small,
-// especially when those locks are contended. The reporting code cannot acquire
-// new locks until the M has released all other locks, which means no memory
-// allocations and encourages use of (temporary) M-local storage.
+// These measurements are subject to some notable constraints: First, the fast
+// path for lock and unlock must remain very fast, with a minimal critical
+// section. Second, the critical section during contention has to remain small
+// too, so low levels of contention are less likely to snowball into large ones.
+// The reporting code cannot acquire new locks until the M has released all
+// other locks, which means no memory allocations and encourages use of
+// (temporary) M-local storage.
//
-// The M will have space for storing one call stack that caused contention, and
-// for the magnitude of that contention. It will also have space to store the
-// magnitude of additional contention the M caused, since it only has space to
-// remember one call stack and might encounter several contention events before
-// it releases all of its locks and is thus able to transfer the local buffer
-// into the profile.
+// The M has space for storing one call stack that caused contention, and the
+// magnitude of that contention. It also has space to store the magnitude of
+// additional contention the M caused, since it might encounter several
+// contention events before it releases all of its locks and is thus able to
+// transfer the locally buffered call stack and magnitude into the profile.
//
-// The M will collect the call stack when it unlocks the contended lock. That
-// minimizes the impact on the critical section of the contended lock, and
-// matches the mutex profile's behavior for contention in sync.Mutex: measured
-// at the Unlock method.
+// The M collects the call stack when it unlocks the contended lock. The
+// traceback takes place outside of the lock's critical section.
//
// The profile for contention on sync.Mutex blames the caller of Unlock for the
// amount of contention experienced by the callers of Lock which had to wait.
// When there are several critical sections, this allows identifying which of
-// them is responsible.
+// them is responsible. We must match that reporting behavior for contention on
+// runtime-internal locks.
//
-// Matching that behavior for runtime-internal locks will require identifying
-// which Ms are blocked on the mutex. The semaphore-based implementation is
-// ready to allow that, but the futex-based implementation will require a bit
-// more work. Until then, we report contention on runtime-internal locks with a
-// call stack taken from the unlock call (like the rest of the user-space
-// "mutex" profile), but assign it a duration value based on how long the
-// previous lock call took (like the user-space "block" profile).
-//
-// Thus, reporting the call stacks of runtime-internal lock contention is
-// guarded by GODEBUG for now. Set GODEBUG=runtimecontentionstacks=1 to enable.
-//
-// TODO(rhysh): plumb through the delay duration, remove GODEBUG, update comment
-//
-// The M will track this by storing a pointer to the lock; lock/unlock pairs for
-// runtime-internal locks are always on the same M.
-//
-// Together, that demands several steps for recording contention. First, when
-// finally acquiring a contended lock, the M decides whether it should plan to
-// profile that event by storing a pointer to the lock in its "to be profiled
-// upon unlock" field. If that field is already set, it uses the relative
-// magnitudes to weight a random choice between itself and the other lock, with
-// the loser's time being added to the "additional contention" field. Otherwise
-// if the M's call stack buffer is occupied, it does the comparison against that
-// sample's magnitude.
-//
-// Second, having unlocked a mutex the M checks to see if it should capture the
-// call stack into its local buffer. Finally, when the M unlocks its last mutex,
-// it transfers the local buffer into the profile. As part of that step, it also
-// transfers any "additional contention" time to the profile. Any lock
-// contention that it experiences while adding samples to the profile will be
-// recorded later as "additional contention" and not include a call stack, to
-// avoid an echo.
-type lockTimer struct {
- lock *mutex
- timeRate int64
- timeStart int64
- tickStart int64
-}
-
-func (lt *lockTimer) begin() {
- rate := int64(atomic.Load64(&mutexprofilerate))
-
- lt.timeRate = gTrackingPeriod
- if rate != 0 && rate < lt.timeRate {
- lt.timeRate = rate
- }
- if int64(cheaprand())%lt.timeRate == 0 {
- lt.timeStart = nanotime()
- }
-
- if rate > 0 && int64(cheaprand())%rate == 0 {
- lt.tickStart = cputicks()
- }
+// When the M unlocks its last mutex, it transfers the locally buffered call
+// stack and magnitude into the profile. As part of that step, it also transfers
+// any "additional contention" time to the profile. Any lock contention that it
+// experiences while adding samples to the profile will be recorded later as
+// "additional contention" and not include a call stack, to avoid an echo.
+type mLockProfile struct {
+ waitTime atomic.Int64 // (nanotime) total time this M has spent waiting in runtime.lockWithRank. Read by runtime/metrics.
+ stack []uintptr // call stack at the point of this M's unlock call, when other Ms had to wait
+ cycles int64 // (cputicks) cycles attributable to "stack"
+ cyclesLost int64 // (cputicks) contention for which we weren't able to record a call stack
+ haveStack bool // stack and cycles are to be added to the mutex profile (even if cycles is 0)
+ disabled bool // attribute all time to "lost"
}
-func (lt *lockTimer) end() {
- gp := getg()
-
- if lt.timeStart != 0 {
- nowTime := nanotime()
- gp.m.mLockProfile.waitTime.Add((nowTime - lt.timeStart) * lt.timeRate)
- }
-
- if lt.tickStart != 0 {
- nowTick := cputicks()
- gp.m.mLockProfile.recordLock(nowTick-lt.tickStart, lt.lock)
+func (prof *mLockProfile) start() int64 {
+ if cheaprandn(gTrackingPeriod) == 0 {
+ return nanotime()
}
+ return 0
}
-type mLockProfile struct {
- waitTime atomic.Int64 // total nanoseconds spent waiting in runtime.lockWithRank
- stack []uintptr // stack that experienced contention in runtime.lockWithRank
- pending uintptr // *mutex that experienced contention (to be traceback-ed)
- cycles int64 // cycles attributable to "pending" (if set), otherwise to "stack"
- cyclesLost int64 // contention for which we weren't able to record a call stack
- haveStack bool // stack and cycles are to be added to the mutex profile
- disabled bool // attribute all time to "lost"
+func (prof *mLockProfile) end(start int64) {
+ if start != 0 {
+ prof.waitTime.Add((nanotime() - start) * gTrackingPeriod)
+ }
}
-func (prof *mLockProfile) recordLock(cycles int64, l *mutex) {
+// recordUnlock prepares data for later addition to the mutex contention
+// profile. The M may hold arbitrary locks during this call.
+//
+// From unlock2, we might not be holding a p in this code.
+//
+//go:nowritebarrierrec
+func (prof *mLockProfile) recordUnlock(cycles int64) {
if cycles < 0 {
cycles = 0
}
return
}
- if uintptr(unsafe.Pointer(l)) == prof.pending {
- // Optimization: we'd already planned to profile this same lock (though
- // possibly from a different unlock site).
- prof.cycles += cycles
- return
- }
-
if prev := prof.cycles; prev > 0 {
// We can only store one call stack for runtime-internal lock contention
// on this M, and we've already got one. Decide which should stay, and
prof.cyclesLost += prev
}
}
- // Saving the *mutex as a uintptr is safe because:
- // - lockrank_on.go does this too, which gives it regular exercise
- // - the lock would only move if it's stack allocated, which means it
- // cannot experience multi-M contention
- prof.pending = uintptr(unsafe.Pointer(l))
+ prof.captureStack()
prof.cycles = cycles
}
-// From unlock2, we might not be holding a p in this code.
-//
-//go:nowritebarrierrec
-func (prof *mLockProfile) recordUnlock(l *mutex) {
- if uintptr(unsafe.Pointer(l)) == prof.pending {
- prof.captureStack()
- }
- if gp := getg(); gp.m.locks == 1 && gp.m.mLockProfile.haveStack {
- prof.store()
- }
-}
-
func (prof *mLockProfile) captureStack() {
if debug.profstackdepth == 0 {
// profstackdepth is set to 0 by the user, so mp.profStack is nil and we
return
}
- skip := 3 // runtime.(*mLockProfile).recordUnlock runtime.unlock2 runtime.unlockWithRank
+ skip := 4 // runtime.(*mLockProfile).recordUnlock runtime.unlock2Wake runtime.unlock2 runtime.unlockWithRank
if staticLockRanking {
// When static lock ranking is enabled, we'll always be on the system
// stack at this point. There will be a runtime.unlockWithRank.func1
// "runtime.unlock".
skip += 1 // runtime.unlockWithRank.func1
}
- prof.pending = 0
prof.haveStack = true
prof.stack[0] = logicalStackSentinel
}
}
+// store adds the M's local record to the mutex contention profile.
+//
+// From unlock2, we might not be holding a p in this code.
+//
+//go:nowritebarrierrec
func (prof *mLockProfile) store() {
+ if gp := getg(); gp.m.locks == 1 && gp.m.mLockProfile.haveStack {
+ prof.storeSlow()
+ }
+}
+
+func (prof *mLockProfile) storeSlow() {
// Report any contention we experience within this function as "lost"; it's
// important that the act of reporting a contention event not lead to a
// reportable contention event. This also means we can use prof.stack