summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/kernel/futex/futex.go
diff options
context:
space:
mode:
authorJamie Liu <jamieliu@google.com>2018-10-08 10:19:27 -0700
committerShentubot <shentubot@google.com>2018-10-08 10:20:38 -0700
commite9e8be661328661b5527f1643727b9a13bbeab48 (patch)
treefb6ab9a8e995d56a73b00c237cb6225799f6417c /pkg/sentry/kernel/futex/futex.go
parent4a00ea557c6e60cdd131b2a9866aa3b0bcb9cb2c (diff)
Implement shared futexes.
- Shared futex objects on shared mappings are represented by Mappable + offset, analogous to Linux's use of inode + offset. Add type futex.Key, and change the futex.Manager bucket API to use futex.Keys instead of addresses. - Extend the futex.Checker interface to be able to return Keys for memory mappings. It returns Keys rather than just mappings because whether the address or the target of the mapping is used in the Key depends on whether the mapping is MAP_SHARED or MAP_PRIVATE; this matters because using mapping target for a futex on a MAP_PRIVATE mapping causes it to stop working across COW-breaking. - futex.Manager.WaitComplete depends on atomic updates to futex.Waiter.addr to determine when it has locked the right bucket, which is much less straightforward for struct futex.Waiter.key. Switch to an atomically-accessed futex.Waiter.bucket pointer. - futex.Manager.Wake now needs to take a futex.Checker to resolve addresses for shared futexes. CLONE_CHILD_CLEARTID requires the exit path to perform a shared futex wakeup (Linux: kernel/fork.c:mm_release() => sys_futex(tsk->clear_child_tid, FUTEX_WAKE, ...)). This is a problem because futexChecker is in the syscalls/linux package. Move it to kernel. PiperOrigin-RevId: 216207039 Change-Id: I708d68e2d1f47e526d9afd95e7fed410c84afccf
Diffstat (limited to 'pkg/sentry/kernel/futex/futex.go')
-rw-r--r--pkg/sentry/kernel/futex/futex.go382
1 files changed, 257 insertions, 125 deletions
diff --git a/pkg/sentry/kernel/futex/futex.go b/pkg/sentry/kernel/futex/futex.go
index 4a1f2a0ef..54b1982a0 100644
--- a/pkg/sentry/kernel/futex/futex.go
+++ b/pkg/sentry/kernel/futex/futex.go
@@ -19,11 +19,78 @@ package futex
import (
"sync"
- "sync/atomic"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/memmap"
"gvisor.googlesource.com/gvisor/pkg/syserror"
)
+// KeyKind indicates the type of a Key.
+type KeyKind int
+
+const (
+ // KindPrivate indicates a private futex (a futex syscall with the
+ // FUTEX_PRIVATE_FLAG set).
+ KindPrivate KeyKind = iota
+
+ // KindSharedPrivate indicates a shared futex on a private memory mapping.
+ // Although KindPrivate and KindSharedPrivate futexes both use memory
+ // addresses to identify futexes, they do not interoperate (in Linux, the
+ // two are distinguished by the FUT_OFF_MMSHARED flag, which is used in key
+ // comparison).
+ KindSharedPrivate
+
+ // KindSharedMappable indicates a shared futex on a memory mapping other
+ // than a private anonymous memory mapping.
+ KindSharedMappable
+)
+
+// Key represents something that a futex waiter may wait on.
+type Key struct {
+ // Kind is the type of the Key.
+ Kind KeyKind
+
+ // Mappable is the memory-mapped object that is represented by the Key.
+ // Mappable is always nil if Kind is not KindSharedMappable, and may be nil
+ // even if it is.
+ Mappable memmap.Mappable
+
+ // MappingIdentity is the MappingIdentity associated with Mappable.
+ // MappingIdentity is always nil is Mappable is nil, and may be nil even if
+ // it isn't.
+ MappingIdentity memmap.MappingIdentity
+
+ // If Kind is KindPrivate or KindSharedPrivate, Offset is the represented
+ // memory address. Otherwise, Offset is the represented offset into
+ // Mappable.
+ Offset uint64
+}
+
+func (k *Key) release() {
+ if k.MappingIdentity != nil {
+ k.MappingIdentity.DecRef()
+ }
+ k.Mappable = nil
+ k.MappingIdentity = nil
+}
+
+func (k *Key) clone() Key {
+ if k.MappingIdentity != nil {
+ k.MappingIdentity.IncRef()
+ }
+ return *k
+}
+
+// Preconditions: k.Kind == KindPrivate or KindSharedPrivate.
+func (k *Key) addr() uintptr {
+ return uintptr(k.Offset)
+}
+
+// matches returns true if a wakeup on k2 should wake a waiter waiting on k.
+func (k *Key) matches(k2 *Key) bool {
+ // k.MappingIdentity is ignored; it's only used for reference counting.
+ return k.Kind == k2.Kind && k.Mappable == k2.Mappable && k.Offset == k2.Offset
+}
+
// Checker abstracts memory accesses. This is useful because the "addresses"
// used in this package may not be real addresses (they could be indices of an
// array, for example), or they could be mapped via some special mechanism.
@@ -41,6 +108,14 @@ type Checker interface {
// Note that op is an opaque operation whose behaviour is defined
// outside of the futex manager.
Op(addr uintptr, op uint32) (bool, error)
+
+ // GetSharedKey returns a Key with kind KindSharedPrivate or
+ // KindSharedMappable corresponding to the memory mapped at address addr.
+ //
+ // If GetSharedKey returns a Key with a non-nil MappingIdentity, a
+ // reference is held on the MappingIdentity, which must be dropped by the
+ // caller when the Key is no longer in use.
+ GetSharedKey(addr uintptr) (Key, error)
}
// Waiter is the struct which gets enqueued into buckets for wake up routines
@@ -53,11 +128,11 @@ type Waiter struct {
// synchronization applies).
//
// - A Waiter is enqueued in a bucket by calling WaitPrepare(). After this,
- // waiterEntry, complete, and addr are protected by the bucket.mu ("bucket
- // lock") of the containing bucket, and bitmask is immutable. complete and
- // addr are additionally mutated using atomic memory operations, ensuring
- // that they can be read using atomic memory operations without holding the
- // bucket lock.
+ // waiterEntry, bucket, and key are protected by the bucket.mu ("bucket
+ // lock") of the containing bucket, and bitmask is immutable. Note that
+ // since bucket is mutated using atomic memory operations, bucket.Load()
+ // may be called without holding the bucket lock, although it may change
+ // racily. See WaitComplete().
//
// - A Waiter is only guaranteed to be no longer queued after calling
// WaitComplete().
@@ -65,15 +140,15 @@ type Waiter struct {
// waiterEntry links Waiter into bucket.waiters.
waiterEntry
- // complete is 1 if the Waiter was removed from its bucket by a wakeup and
- // 0 otherwise.
- complete int32
+ // bucket is the bucket this waiter is queued in. If bucket is nil, the
+ // waiter is not waiting and is not in any bucket.
+ bucket AtomicPtrBucket
// C is sent to when the Waiter is woken.
C chan struct{}
- // addr is the address being waited on.
- addr uintptr
+ // key is what this waiter is waiting on.
+ key Key
// The bitmask we're waiting on.
// This is used the case of a FUTEX_WAKE_BITSET.
@@ -87,7 +162,14 @@ func NewWaiter() *Waiter {
}
}
+// woken returns true if w has been woken since the last call to WaitPrepare.
+func (w *Waiter) woken() bool {
+ return len(w.C) != 0
+}
+
// bucket holds a list of waiters for a given address hash.
+//
+// +stateify savable
type bucket struct {
// mu protects waiters and contained Waiter state. See comment in Waiter.
mu sync.Mutex `state:"nosave"`
@@ -99,10 +181,10 @@ type bucket struct {
// bucket and returns the number of waiters woken.
//
// Preconditions: b.mu must be locked.
-func (b *bucket) wakeLocked(addr uintptr, bitmask uint32, n int) int {
+func (b *bucket) wakeLocked(key *Key, bitmask uint32, n int) int {
done := 0
for w := b.waiters.Front(); done < n && w != nil; {
- if w.addr != addr || w.bitmask&bitmask == 0 {
+ if !w.key.matches(key) || w.bitmask&bitmask == 0 {
// Not matching.
w = w.Next()
continue
@@ -114,15 +196,15 @@ func (b *bucket) wakeLocked(addr uintptr, bitmask uint32, n int) int {
b.waiters.Remove(woke)
woke.C <- struct{}{}
- // NOTE: The above channel write establishes a write barrier
- // according to the memory model, so nothing may be ordered
- // around it. Since we've dequeued w and will never touch it
- // again, we can safely store 1 to w.complete here and allow
- // the WaitComplete() to short-circuit grabbing the bucket
- // lock. If they somehow miss the w.complete, we are still
- // holding the lock, so we can know that they won't dequeue w,
- // assume it's free and have the below operation afterwards.
- atomic.StoreInt32(&woke.complete, 1)
+ // NOTE: The above channel write establishes a write barrier according
+ // to the memory model, so nothing may be ordered around it. Since
+ // we've dequeued woke and will never touch it again, we can safely
+ // store nil to woke.bucket here and allow the WaitComplete() to
+ // short-circuit grabbing the bucket lock. If they somehow miss the
+ // store, we are still holding the lock, so we can know that they won't
+ // dequeue woke, assume it's free and have the below operation
+ // afterwards.
+ woke.bucket.Store(nil)
done++
}
return done
@@ -132,10 +214,10 @@ func (b *bucket) wakeLocked(addr uintptr, bitmask uint32, n int) int {
// bucket "to".
//
// Preconditions: b and to must be locked.
-func (b *bucket) requeueLocked(to *bucket, addr, naddr uintptr, n int) int {
+func (b *bucket) requeueLocked(to *bucket, key, nkey *Key, n int) int {
done := 0
for w := b.waiters.Front(); done < n && w != nil; {
- if w.addr != addr {
+ if !w.key.matches(key) {
// Not matching.
w = w.Next()
continue
@@ -144,8 +226,10 @@ func (b *bucket) requeueLocked(to *bucket, addr, naddr uintptr, n int) int {
requeued := w
w = w.Next() // Next iteration.
b.waiters.Remove(requeued)
- atomic.StoreUintptr(&requeued.addr, naddr)
+ requeued.key.release()
+ requeued.key = nkey.clone()
to.waiters.PushBack(requeued)
+ requeued.bucket.Store(to)
done++
}
return done
@@ -158,19 +242,22 @@ const (
bucketCountBits = 10
)
-func checkAddr(addr uintptr) error {
+// getKey returns a Key representing address addr in c.
+func getKey(c Checker, addr uintptr, private bool) (Key, error) {
// Ensure the address is aligned.
// It must be a DWORD boundary.
if addr&0x3 != 0 {
- return syserror.EINVAL
+ return Key{}, syserror.EINVAL
}
-
- return nil
+ if private {
+ return Key{Kind: KindPrivate, Offset: uint64(addr)}, nil
+ }
+ return c.GetSharedKey(addr)
}
// bucketIndexForAddr returns the index into Manager.buckets for addr.
func bucketIndexForAddr(addr uintptr) uintptr {
- // - The bottom 2 bits of addr must be 0, per checkAddr.
+ // - The bottom 2 bits of addr must be 0, per getKey.
//
// - On amd64, the top 16 bits of addr (bits 48-63) must be equal to bit 47
// for a canonical address, and (on all existing platforms) bit 47 must be
@@ -199,171 +286,216 @@ func bucketIndexForAddr(addr uintptr) uintptr {
//
// +stateify savable
type Manager struct {
- buckets [bucketCount]bucket `state:"zerovalue"`
+ // privateBuckets holds buckets for KindPrivate and KindSharedPrivate
+ // futexes.
+ privateBuckets [bucketCount]bucket `state:"zerovalue"`
+
+ // sharedBucket is the bucket for KindSharedMappable futexes. sharedBucket
+ // may be shared by multiple Managers. The sharedBucket pointer is
+ // immutable.
+ sharedBucket *bucket
}
// NewManager returns an initialized futex manager.
-// N.B. we use virtual address to tag futexes, so it only works for private
-// (within a single process) futex.
func NewManager() *Manager {
- return &Manager{}
+ return &Manager{
+ sharedBucket: &bucket{},
+ }
}
-// lockBucket returns a locked bucket for the given addr.
-//
-// Preconditions: checkAddr(addr) == nil.
-func (m *Manager) lockBucket(addr uintptr) *bucket {
- b := &m.buckets[bucketIndexForAddr(addr)]
+// Fork returns a new Manager. Shared futex clients using the returned Manager
+// may interoperate with those using m.
+func (m *Manager) Fork() *Manager {
+ return &Manager{
+ sharedBucket: m.sharedBucket,
+ }
+}
+
+// lockBucket returns a locked bucket for the given key.
+func (m *Manager) lockBucket(k *Key) *bucket {
+ var b *bucket
+ if k.Kind == KindSharedMappable {
+ b = m.sharedBucket
+ } else {
+ b = &m.privateBuckets[bucketIndexForAddr(k.addr())]
+ }
b.mu.Lock()
return b
}
-// lockBuckets returns locked buckets for the given addrs.
-//
-// Preconditions: checkAddr(addr1) == checkAddr(addr2) == nil.
-func (m *Manager) lockBuckets(addr1 uintptr, addr2 uintptr) (*bucket, *bucket) {
- i1 := bucketIndexForAddr(addr1)
- i2 := bucketIndexForAddr(addr2)
- b1 := &m.buckets[i1]
- b2 := &m.buckets[i2]
-
- // Ensure that buckets are locked in a consistent order (lowest index
- // first) to avoid circular locking.
- switch {
- case i1 < i2:
- b1.mu.Lock()
- b2.mu.Lock()
- case i2 < i1:
- b2.mu.Lock()
- b1.mu.Lock()
- default:
- b1.mu.Lock()
+// lockBuckets returns locked buckets for the given keys.
+func (m *Manager) lockBuckets(k1, k2 *Key) (*bucket, *bucket) {
+ // Buckets must be consistently ordered to avoid circular lock
+ // dependencies. We order buckets in m.privateBuckets by index (lowest
+ // index first), and all buckets in m.privateBuckets precede
+ // m.sharedBucket.
+
+ // Handle the common case first:
+ if k1.Kind != KindSharedMappable && k2.Kind != KindSharedMappable {
+ i1 := bucketIndexForAddr(k1.addr())
+ i2 := bucketIndexForAddr(k2.addr())
+ b1 := &m.privateBuckets[i1]
+ b2 := &m.privateBuckets[i2]
+ switch {
+ case i1 < i2:
+ b1.mu.Lock()
+ b2.mu.Lock()
+ case i2 < i1:
+ b2.mu.Lock()
+ b1.mu.Lock()
+ default:
+ b1.mu.Lock()
+ }
+ return b1, b2
}
+ // At least one of b1 or b2 should be m.sharedBucket.
+ b1 := m.sharedBucket
+ b2 := m.sharedBucket
+ if k1.Kind != KindSharedMappable {
+ b1 = m.lockBucket(k1)
+ } else if k2.Kind != KindSharedMappable {
+ b2 = m.lockBucket(k2)
+ }
+ m.sharedBucket.mu.Lock()
return b1, b2
}
// Wake wakes up to n waiters matching the bitmask on the given addr.
// The number of waiters woken is returned.
-func (m *Manager) Wake(addr uintptr, bitmask uint32, n int) (int, error) {
- if err := checkAddr(addr); err != nil {
+func (m *Manager) Wake(c Checker, addr uintptr, private bool, bitmask uint32, n int) (int, error) {
+ // This function is very hot; avoid defer.
+ k, err := getKey(c, addr, private)
+ if err != nil {
return 0, err
}
- b := m.lockBucket(addr)
- // This function is very hot; avoid defer.
- r := b.wakeLocked(addr, bitmask, n)
+ b := m.lockBucket(&k)
+ r := b.wakeLocked(&k, bitmask, n)
+
b.mu.Unlock()
+ k.release()
return r, nil
}
-func (m *Manager) doRequeue(c Checker, addr uintptr, val uint32, naddr uintptr, nwake int, nreq int) (int, error) {
- if err := checkAddr(addr); err != nil {
+func (m *Manager) doRequeue(c Checker, addr, naddr uintptr, private bool, checkval bool, val uint32, nwake int, nreq int) (int, error) {
+ k1, err := getKey(c, addr, private)
+ if err != nil {
return 0, err
}
- if err := checkAddr(naddr); err != nil {
+ defer k1.release()
+ k2, err := getKey(c, naddr, private)
+ if err != nil {
return 0, err
}
+ defer k2.release()
- b1, b2 := m.lockBuckets(addr, naddr)
+ b1, b2 := m.lockBuckets(&k1, &k2)
defer b1.mu.Unlock()
if b2 != b1 {
defer b2.mu.Unlock()
}
- // Check our value.
- // This only applied for RequeueCmp().
- if c != nil {
+ if checkval {
if err := c.Check(addr, val); err != nil {
return 0, err
}
}
// Wake the number required.
- done := b1.wakeLocked(addr, ^uint32(0), nwake)
+ done := b1.wakeLocked(&k1, ^uint32(0), nwake)
// Requeue the number required.
- b1.requeueLocked(b2, addr, naddr, nreq)
+ b1.requeueLocked(b2, &k1, &k2, nreq)
return done, nil
}
// Requeue wakes up to nwake waiters on the given addr, and unconditionally
// requeues up to nreq waiters on naddr.
-func (m *Manager) Requeue(addr uintptr, naddr uintptr, nwake int, nreq int) (int, error) {
- return m.doRequeue(nil, addr, 0, naddr, nwake, nreq)
+func (m *Manager) Requeue(c Checker, addr, naddr uintptr, private bool, nwake int, nreq int) (int, error) {
+ return m.doRequeue(c, addr, naddr, private, false, 0, nwake, nreq)
}
// RequeueCmp atomically checks that the addr contains val (via the Checker),
// wakes up to nwake waiters on addr and then unconditionally requeues nreq
// waiters on naddr.
-func (m *Manager) RequeueCmp(c Checker, addr uintptr, val uint32, naddr uintptr, nwake int, nreq int) (int, error) {
- return m.doRequeue(c, addr, val, naddr, nwake, nreq)
+func (m *Manager) RequeueCmp(c Checker, addr, naddr uintptr, private bool, val uint32, nwake int, nreq int) (int, error) {
+ return m.doRequeue(c, addr, naddr, private, true, val, nwake, nreq)
}
// WakeOp atomically applies op to the memory address addr2, wakes up to nwake1
// waiters unconditionally from addr1, and, based on the original value at addr2
// and a comparison encoded in op, wakes up to nwake2 waiters from addr2.
// It returns the total number of waiters woken.
-func (m *Manager) WakeOp(c Checker, addr1 uintptr, addr2 uintptr, nwake1 int, nwake2 int, op uint32) (int, error) {
- if err := checkAddr(addr1); err != nil {
+func (m *Manager) WakeOp(c Checker, addr1, addr2 uintptr, private bool, nwake1 int, nwake2 int, op uint32) (int, error) {
+ k1, err := getKey(c, addr1, private)
+ if err != nil {
return 0, err
}
- if err := checkAddr(addr2); err != nil {
+ defer k1.release()
+ k2, err := getKey(c, addr2, private)
+ if err != nil {
return 0, err
}
+ defer k2.release()
- b1, b2 := m.lockBuckets(addr1, addr2)
+ b1, b2 := m.lockBuckets(&k1, &k2)
+ defer b1.mu.Unlock()
+ if b2 != b1 {
+ defer b2.mu.Unlock()
+ }
done := 0
cond, err := c.Op(addr2, op)
- if err == nil {
- // Wake up up to nwake1 entries from the first bucket.
- done = b1.wakeLocked(addr1, ^uint32(0), nwake1)
-
- // Wake up up to nwake2 entries from the second bucket if the
- // operation yielded true.
- if cond {
- done += b2.wakeLocked(addr2, ^uint32(0), nwake2)
- }
+ if err != nil {
+ return 0, err
}
- b1.mu.Unlock()
- if b2 != b1 {
- b2.mu.Unlock()
+ // Wake up up to nwake1 entries from the first bucket.
+ done = b1.wakeLocked(&k1, ^uint32(0), nwake1)
+
+ // Wake up up to nwake2 entries from the second bucket if the
+ // operation yielded true.
+ if cond {
+ done += b2.wakeLocked(&k2, ^uint32(0), nwake2)
}
- return done, err
+
+ return done, nil
}
// WaitPrepare atomically checks that addr contains val (via the Checker), then
// enqueues w to be woken by a send to w.C. If WaitPrepare returns nil, the
// Waiter must be subsequently removed by calling WaitComplete, whether or not
// a wakeup is received on w.C.
-func (m *Manager) WaitPrepare(w *Waiter, c Checker, addr uintptr, val uint32, bitmask uint32) error {
- if err := checkAddr(addr); err != nil {
+func (m *Manager) WaitPrepare(w *Waiter, c Checker, addr uintptr, private bool, val uint32, bitmask uint32) error {
+ k, err := getKey(c, addr, private)
+ if err != nil {
return err
}
+ // Ownership of k is transferred to w below.
// Prepare the Waiter before taking the bucket lock.
- w.complete = 0
select {
case <-w.C:
default:
}
- w.addr = addr
+ w.key = k
w.bitmask = bitmask
- b := m.lockBucket(addr)
+ b := m.lockBucket(&k)
// This function is very hot; avoid defer.
// Perform our atomic check.
if err := c.Check(addr, val); err != nil {
b.mu.Unlock()
+ w.key.release()
return err
}
// Add the waiter to the bucket.
b.waiters.PushBack(w)
+ w.bucket.Store(b)
b.mu.Unlock()
return nil
@@ -372,36 +504,36 @@ func (m *Manager) WaitPrepare(w *Waiter, c Checker, addr uintptr, val uint32, bi
// WaitComplete must be called when a Waiter previously added by WaitPrepare is
// no longer eligible to be woken.
func (m *Manager) WaitComplete(w *Waiter) {
- // Can we short-circuit acquiring the lock?
- // This is the happy path where a notification
- // was received and we don't need to dequeue this
- // waiter from any list (or take any locks).
- if atomic.LoadInt32(&w.complete) != 0 {
- return
- }
-
- // Take the bucket lock. Note that without holding the bucket lock, the
- // waiter is not guaranteed to stay in that bucket, so after we take the
- // bucket lock, we must ensure that the bucket hasn't changed: if it
- // happens to have changed, we release the old bucket lock and try again
- // with the new bucket; if it hasn't changed, we know it won't change now
- // because we hold the lock.
- var b *bucket
+ // Remove w from the bucket it's in.
for {
- addr := atomic.LoadUintptr(&w.addr)
- b = m.lockBucket(addr)
- // We still have to use an atomic load here, because if w was racily
- // requeued then w.addr is not protected by b.mu.
- if addr == atomic.LoadUintptr(&w.addr) {
+ b := w.bucket.Load()
+
+ // If b is nil, the waiter isn't in any bucket anymore. This can't be
+ // racy because the waiter can't be concurrently re-queued in another
+ // bucket.
+ if b == nil {
break
}
- b.mu.Unlock()
- }
- // Remove waiter from the bucket. w.complete can only be stored with b.mu
- // locked, so this load doesn't need to use sync/atomic.
- if w.complete == 0 {
+ // Take the bucket lock. Note that without holding the bucket lock, the
+ // waiter is not guaranteed to stay in that bucket, so after we take
+ // the bucket lock, we must ensure that the bucket hasn't changed: if
+ // it happens to have changed, we release the old bucket lock and try
+ // again with the new bucket; if it hasn't changed, we know it won't
+ // change now because we hold the lock.
+ b.mu.Lock()
+ if b != w.bucket.Load() {
+ b.mu.Unlock()
+ continue
+ }
+
+ // Remove w from b.
b.waiters.Remove(w)
+ w.bucket.Store(nil)
+ b.mu.Unlock()
+ break
}
- b.mu.Unlock()
+
+ // Release references held by the waiter.
+ w.key.release()
}