Skip to content

Commit

Permalink
Merge #39609
Browse files Browse the repository at this point in the history
39609: syncutil: add AssertRHeld methods to RWMutex implementations r=nvanbenschoten a=nvanbenschoten

This mirrors the AssertHeld methods we have on these mutexes and extends them for read locks. These serve as both a useful assertion and good documentation (because they compile away for real builds).

#39119 reminded me that I had been sitting on this commit since June as part of a larger change that won't be making it in any time soon.

Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Aug 13, 2019
2 parents 7beb89f + 86241f9 commit 3b316ba
Show file tree
Hide file tree
Showing 6 changed files with 121 additions and 42 deletions.
5 changes: 2 additions & 3 deletions pkg/storage/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"errors"
"fmt"
"reflect"
"testing"

"github.com/cockroachdb/cockroach/pkg/config"
Expand Down Expand Up @@ -192,7 +191,7 @@ func TestStoreGossipSystemData(t *testing.T) {
t.Fatal(err)
}
testutils.SucceedsSoon(t, func() error {
if !reflect.DeepEqual(getSystemConfig(), config.NewSystemConfig(sc.DefaultZoneConfig)) {
if !getSystemConfig().DefaultZoneConfig.Equal(sc.DefaultZoneConfig) {
return errors.New("system config not empty")
}
if getNodeLiveness() != (storagepb.Liveness{}) {
Expand All @@ -205,7 +204,7 @@ func TestStoreGossipSystemData(t *testing.T) {
// data is gossiped.
mtc.restartStore(0)
testutils.SucceedsSoon(t, func() error {
if reflect.DeepEqual(getSystemConfig(), config.NewSystemConfig(sc.DefaultZoneConfig)) {
if !getSystemConfig().DefaultZoneConfig.Equal(sc.DefaultZoneConfig) {
return errors.New("system config not gossiped")
}
if getNodeLiveness() == (storagepb.Liveness{}) {
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -667,6 +667,7 @@ func (r *Replica) Desc() *roachpb.RangeDescriptor {
}

func (r *Replica) descRLocked() *roachpb.RangeDescriptor {
r.mu.AssertRHeld()
return r.mu.state.Desc
}

Expand Down
6 changes: 5 additions & 1 deletion pkg/util/syncutil/mutex_deadlock.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,5 +37,9 @@ type RWMutex struct {
}

// AssertHeld is a no-op for deadlock mutexes.
func (m *RWMutex) AssertHeld() {
func (rw *RWMutex) AssertHeld() {
}

// AssertRHeld is a no-op for deadlock mutexes.
func (rw *RWMutex) AssertRHeld() {
}
22 changes: 18 additions & 4 deletions pkg/util/syncutil/mutex_sync.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,22 @@ type RWMutex struct {
// particular lock may use this to enforce this requirement more directly than
// relying on the race detector.
//
// Note that we do not require the lock to be held by any particular thread,
// just that some thread holds the lock. This is both more efficient and allows
// for rare cases where a mutex is locked in one thread and used in another.
func (m *RWMutex) AssertHeld() {
// Note that we do not require the exclusive lock to be held by any particular
// thread, just that some thread holds the lock. This is both more efficient
// and allows for rare cases where a mutex is locked in one thread and used in
// another.
func (rw *RWMutex) AssertHeld() {
}

// AssertRHeld may panic if the mutex is not locked for reading (but it is not
// required to do so). If the mutex is locked for writing, it is also considered
// to be locked for reading. Functions which require that their callers hold a
// particular lock may use this to enforce this requirement more directly than
// relying on the race detector.
//
// Note that we do not require the shared lock to be held by any particular
// thread, just that some thread holds the lock. This is both more efficient
// and allows for rare cases where a mutex is locked in one thread and used in
// another.
func (rw *RWMutex) AssertRHeld() {
}
87 changes: 64 additions & 23 deletions pkg/util/syncutil/mutex_sync_race.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,19 +20,19 @@ import (

// A Mutex is a mutual exclusion lock.
type Mutex struct {
mu sync.Mutex
isLocked int32 // updated atomically
mu sync.Mutex
wLocked int32 // updated atomically
}

// Lock implements sync.Locker.
// Lock locks m.
func (m *Mutex) Lock() {
m.mu.Lock()
atomic.StoreInt32(&m.isLocked, 1)
atomic.StoreInt32(&m.wLocked, 1)
}

// Unlock implements sync.Locker.
// Unlock unlocks m.
func (m *Mutex) Unlock() {
atomic.StoreInt32(&m.isLocked, 0)
atomic.StoreInt32(&m.wLocked, 0)
m.mu.Unlock()
}

Expand All @@ -45,39 +45,80 @@ func (m *Mutex) Unlock() {
// just that some thread holds the lock. This is both more efficient and allows
// for rare cases where a mutex is locked in one thread and used in another.
func (m *Mutex) AssertHeld() {
if atomic.LoadInt32(&m.isLocked) == 0 {
panic("mutex is not locked")
if atomic.LoadInt32(&m.wLocked) == 0 {
panic("mutex is not write locked")
}
}

// An RWMutex is a reader/writer mutual exclusion lock.
type RWMutex struct {
sync.RWMutex
isLocked int32 // updated atomically
wLocked int32 // updated atomically
rLocked int32 // updated atomically
}

// Lock implements sync.Locker.
func (m *RWMutex) Lock() {
m.RWMutex.Lock()
atomic.StoreInt32(&m.isLocked, 1)
// Lock locks rw for writing.
func (rw *RWMutex) Lock() {
rw.RWMutex.Lock()
atomic.StoreInt32(&rw.wLocked, 1)
}

// Unlock implements sync.Locker.
func (m *RWMutex) Unlock() {
atomic.StoreInt32(&m.isLocked, 0)
m.RWMutex.Unlock()
// Unlock unlocks rw for writing.
func (rw *RWMutex) Unlock() {
atomic.StoreInt32(&rw.wLocked, 0)
rw.RWMutex.Unlock()
}

// RLock locks m for reading.
func (rw *RWMutex) RLock() {
rw.RWMutex.RLock()
atomic.AddInt32(&rw.rLocked, 1)
}

// RUnlock undoes a single RLock call.
func (rw *RWMutex) RUnlock() {
atomic.AddInt32(&rw.rLocked, -1)
rw.RWMutex.RUnlock()
}

// RLocker returns a Locker interface that implements
// the Lock and Unlock methods by calling rw.RLock and rw.RUnlock.
func (rw *RWMutex) RLocker() sync.Locker {
return (*rlocker)(rw)
}

type rlocker RWMutex

func (r *rlocker) Lock() { (*RWMutex)(r).RLock() }
func (r *rlocker) Unlock() { (*RWMutex)(r).RUnlock() }

// AssertHeld may panic if the mutex is not locked for writing (but it is not
// required to do so). Functions which require that their callers hold a
// particular lock may use this to enforce this requirement more directly than
// relying on the race detector.
//
// Note that we do not require the lock to be held by any particular thread,
// just that some thread holds the lock. This is both more efficient and allows
// for rare cases where a mutex is locked in one thread and used in another.
func (m *RWMutex) AssertHeld() {
if atomic.LoadInt32(&m.isLocked) == 0 {
panic("mutex is not locked")
// Note that we do not require the exclusive lock to be held by any particular
// thread, just that some thread holds the lock. This is both more efficient
// and allows for rare cases where a mutex is locked in one thread and used in
// another.
func (rw *RWMutex) AssertHeld() {
if atomic.LoadInt32(&rw.wLocked) == 0 {
panic("mutex is not write locked")
}
}

// AssertRHeld may panic if the mutex is not locked for reading (but it is not
// required to do so). If the mutex is locked for writing, it is also considered
// to be locked for reading. Functions which require that their callers hold a
// particular lock may use this to enforce this requirement more directly than
// relying on the race detector.
//
// Note that we do not require the shared lock to be held by any particular
// thread, just that some thread holds the lock. This is both more efficient
// and allows for rare cases where a mutex is locked in one thread and used in
// another.
func (rw *RWMutex) AssertRHeld() {
if atomic.LoadInt32(&rw.wLocked) == 0 && atomic.LoadInt32(&rw.rLocked) == 0 {
panic("mutex is not read locked")
}
}
42 changes: 31 additions & 11 deletions pkg/util/syncutil/mutex_sync_race_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,11 @@

package syncutil

import "testing"
import (
"testing"

"github.com/stretchr/testify/require"
)

func TestAssertHeld(t *testing.T) {
type mutex interface {
Expand All @@ -34,15 +38,31 @@ func TestAssertHeld(t *testing.T) {
c.m.AssertHeld()
c.m.Unlock()

func() {
defer func() {
if r := recover(); r == nil {
t.Fatal("did not get expected panic")
} else if a, e := r.(string), "mutex is not locked"; a != e {
t.Fatalf("got %q, expected %q", a, e)
}
}()
c.m.AssertHeld()
}()
// The unsuccessful case.
require.PanicsWithValue(t, "mutex is not write locked", c.m.AssertHeld)
}
}

func TestAssertRHeld(t *testing.T) {
var m RWMutex

// The normal, successful case.
m.RLock()
m.AssertRHeld()
m.RUnlock()

// The normal case with two readers.
m.RLock()
m.RLock()
m.AssertRHeld()
m.RUnlock()
m.RUnlock()

// The case where a write lock is held.
m.Lock()
m.AssertRHeld()
m.Unlock()

// The unsuccessful case with no readers.
require.PanicsWithValue(t, "mutex is not read locked", m.AssertRHeld)
}

0 comments on commit 3b316ba

Please sign in to comment.