Skip to content

Commit 07d2b63

Browse files
committed
WIP
An experiment using drwmutex [1] to speed up read lock contention on 96 vCPUs, as observed in [2]. The final run of `kv95/enc=false/nodes=3/cpu=96` exhibited average throughput of 173413 ops/sec. That's worse than the implementation without RWMutex. It appears that read lock, as implemented by Go's runtime scales poorly to a high number of vCPUs [3]. On the other hand, the write lock under drwmutex requires acquiring 96 locks in this case, which appears to be the only bottleneck; the sharded read lock is optimal enough that it doesn't show up on the cpu profile. The only slow down appears to be the write lock inside getStatsForStmtWithKeySlow which is unavoidable. Although inconclusive, it appears that drwmutex doesn't scale well above a certain number of vCPUs, when the write mutex is on a critical path. [1] https://github.com/jonhoo/drwmutex [2] cockroachdb#109443 [3] golang/go#17973 Epic: none Release note: None
1 parent 31dc96b commit 07d2b63

File tree

13 files changed

+277
-88
lines changed

13 files changed

+277
-88
lines changed

pkg/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -2231,6 +2231,7 @@ GO_TARGETS = [
22312231
"//pkg/util/ctxutil:ctxutil",
22322232
"//pkg/util/ctxutil:ctxutil_test",
22332233
"//pkg/util/debugutil:debugutil",
2234+
"//pkg/util/drwmutex:drwmutex",
22342235
"//pkg/util/duration:duration",
22352236
"//pkg/util/duration:duration_test",
22362237
"//pkg/util/encoding/csv:csv",

pkg/cmd/roachtest/tests/kv.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -142,7 +142,7 @@ func registerKV(r registry.Registry) {
142142
concurrency := ifLocal(c, "", " --concurrency="+fmt.Sprint(computeConcurrency(opts)))
143143
splits := " --splits=" + strconv.Itoa(computeNumSplits(opts))
144144
if opts.duration == 0 {
145-
opts.duration = 30 * time.Minute
145+
opts.duration = 7 * time.Minute
146146
}
147147
duration := " --duration=" + ifLocal(c, "10s", opts.duration.String())
148148
var readPercent string

pkg/sql/sqlstats/ssmemstorage/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ go_library(
2020
"//pkg/sql/sqlstats",
2121
"//pkg/sql/sqlstats/insights",
2222
"//pkg/util",
23+
"//pkg/util/drwmutex",
2324
"//pkg/util/log",
2425
"//pkg/util/mon",
2526
"//pkg/util/syncutil",

pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go

+6-4
Original file line numberDiff line numberDiff line change
@@ -36,8 +36,9 @@ func NewStmtStatsIterator(
3636
) StmtStatsIterator {
3737
var stmtKeys stmtList
3838
func() {
39-
container.mu.RLock()
40-
defer container.mu.RUnlock()
39+
rlock := container.mu.mx.RLocker()
40+
rlock.Lock()
41+
defer rlock.Unlock()
4142
for k := range container.mu.stmts {
4243
stmtKeys = append(stmtKeys, k)
4344
}
@@ -129,11 +130,12 @@ type TxnStatsIterator struct {
129130
// NewTxnStatsIterator returns a new instance of TxnStatsIterator.
130131
func NewTxnStatsIterator(container *Container, options sqlstats.IteratorOptions) TxnStatsIterator {
131132
var txnKeys txnList
132-
container.mu.Lock()
133+
rlock := container.mu.mx.RLocker()
134+
rlock.Lock()
133135
for k := range container.mu.txns {
134136
txnKeys = append(txnKeys, k)
135137
}
136-
container.mu.Unlock()
138+
rlock.Unlock()
137139
if options.SortedKey {
138140
sort.Sort(txnKeys)
139141
}

pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go

+75-74
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"encoding/json"
2020
"fmt"
2121
"sync/atomic"
22+
"sync"
2223
"time"
2324
"unsafe"
2425

@@ -33,6 +34,7 @@ import (
3334
"github.com/cockroachdb/cockroach/pkg/util/mon"
3435
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
3536
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
37+
"github.com/cockroachdb/cockroach/pkg/util/drwmutex"
3638
"github.com/cockroachdb/errors"
3739
)
3840

@@ -95,29 +97,32 @@ type Container struct {
9597
}
9698

9799
mu struct {
98-
syncutil.RWMutex
99-
100-
// acc is the memory account that tracks memory allocations related to stmts
101-
// and txns within this Container struct.
102-
// Since currently we do not destroy the Container struct when we perform
103-
// reset, we never close this account.
104-
acc mon.BoundAccount
100+
mx drwmutex.DRWMutex
105101

106102
stmts map[stmtKey]*stmtStats
107103
txns map[appstatspb.TransactionFingerprintID]*txnStats
108-
}
109104

110105
// Use a separate lock to avoid lock contention. Don't block the statement
111106
// stats just to update the sampled plan time.
112-
muPlanCache struct {
113-
syncutil.RWMutex
107+
//muPlanCache struct {
108+
// mx drwmutex.DRWMutex
114109

115110
// sampledPlanMetadataCache records when was the last time the plan was
116111
// sampled. This data structure uses a subset of stmtKey as the key into
117112
// in-memory dictionary in order to allow lookup for whether a plan has been
118113
// sampled for a statement without needing to know the statement's
119114
// transaction fingerprintID.
120115
sampledPlanMetadataCache map[sampledPlanKey]time.Time
116+
//}
117+
}
118+
119+
muAcc struct {
120+
sync.Mutex
121+
// acc is the memory account that tracks memory allocations related to stmts
122+
// and txns within this Container struct.
123+
// Since currently we do not destroy the Container struct when we perform
124+
// reset, we never close this account.
125+
acc mon.BoundAccount
121126
}
122127

123128
txnCounts transactionCounts
@@ -155,12 +160,13 @@ func New(
155160
}
156161

157162
if mon != nil {
158-
s.mu.acc = mon.MakeBoundAccount()
163+
s.muAcc.acc = mon.MakeBoundAccount()
159164
}
160165

166+
s.mu.mx = drwmutex.New()
161167
s.mu.stmts = make(map[stmtKey]*stmtStats)
162168
s.mu.txns = make(map[appstatspb.TransactionFingerprintID]*txnStats)
163-
s.muPlanCache.sampledPlanMetadataCache = make(map[sampledPlanKey]time.Time)
169+
s.mu.sampledPlanMetadataCache = make(map[sampledPlanKey]time.Time)
164170

165171
s.atomic.uniqueStmtFingerprintCount = uniqueStmtFingerprintCount
166172
s.atomic.uniqueTxnFingerprintCount = uniqueTxnFingerprintCount
@@ -276,7 +282,7 @@ func NewTempContainerFromExistingStmtStats(
276282
transactionFingerprintID: statistics[i].Key.KeyData.TransactionFingerprintID,
277283
}
278284
stmtStats, _, throttled :=
279-
container.getStatsForStmtWithKeyLocked(key, statistics[i].ID, true /* createIfNonexistent */)
285+
container.getStatsForStmtWithKey(key, statistics[i].ID, true /* createIfNonexistent */)
280286
if throttled {
281287
return nil /* container */, nil /* remaining */, ErrFingerprintLimitReached
282288
}
@@ -360,8 +366,9 @@ func (s *Container) NewApplicationStatsWithInheritedOptions() sqlstats.Applicati
360366
uniqueStmtFingerprintCount int64
361367
uniqueTxnFingerprintCount int64
362368
)
363-
s.mu.Lock()
364-
defer s.mu.Unlock()
369+
rlock := s.mu.mx.RLocker()
370+
rlock.Lock()
371+
defer rlock.Unlock()
365372
return New(
366373
s.st,
367374
sqlstats.MaxSQLStatsStmtFingerprintsPerExplicitTxn,
@@ -552,31 +559,23 @@ func (s *Container) getStatsForStmtWithKey(
552559
) (stats *stmtStats, created, throttled bool) {
553560
// Use the read lock to get the key to avoid contention.
554561
ok := func() (ok bool) {
555-
s.mu.RLock()
556-
defer s.mu.RUnlock()
562+
rlock := s.mu.mx.RLocker()
563+
rlock.Lock()
557564
stats, ok = s.mu.stmts[key]
565+
rlock.Unlock()
566+
558567
return ok
559568
}()
560569
if ok || !createIfNonexistent {
561-
return stats, false /* created */, false /* throttled */
562-
}
563-
564-
// Key does not exist in map. Take a full lock to add the key.
565-
s.mu.Lock()
566-
defer s.mu.Unlock()
567-
return s.getStatsForStmtWithKeyLocked(key, stmtFingerprintID, createIfNonexistent)
570+
return stats, false /* created */, false /* throttled */
571+
}
572+
// Key does not exist in map, slow path.
573+
return s.getStatsForStmtWithKeySlow(key, stmtFingerprintID, createIfNonexistent)
568574
}
569575

570-
func (s *Container) getStatsForStmtWithKeyLocked(
576+
func (s *Container) getStatsForStmtWithKeySlow(
571577
key stmtKey, stmtFingerprintID appstatspb.StmtFingerprintID, createIfNonexistent bool,
572578
) (stats *stmtStats, created, throttled bool) {
573-
// Retrieve the per-statement statistic object, and create it if it
574-
// doesn't exist yet.
575-
stats, ok := s.mu.stmts[key]
576-
if ok || !createIfNonexistent {
577-
return stats, false /* created */, false /* throttled */
578-
}
579-
580579
// If the uniqueStmtFingerprintCount is nil, then we don't check for
581580
// fingerprint limit.
582581
if s.atomic.uniqueStmtFingerprintCount != nil {
@@ -594,9 +593,12 @@ func (s *Container) getStatsForStmtWithKeyLocked(
594593
}
595594
stats = &stmtStats{}
596595
stats.ID = stmtFingerprintID
597-
s.mu.stmts[key] = stats
596+
t := s.getTimeNow()
598597

599-
s.setLogicalPlanLastSampled(key.sampledPlanKey, s.getTimeNow())
598+
s.mu.mx.Lock()
599+
s.mu.stmts[key] = stats
600+
s.mu.sampledPlanMetadataCache[key.sampledPlanKey] = t
601+
s.mu.mx.Unlock()
600602

601603
return stats, true /* created */, false /* throttled */
602604
}
@@ -608,18 +610,19 @@ func (s *Container) getStatsForTxnWithKey(
608610
) (stats *txnStats, created, throttled bool) {
609611
// Use the read lock to get the key to avoid contention
610612
ok := func() (ok bool) {
611-
s.mu.RLock()
612-
defer s.mu.RUnlock()
613+
rlock := s.mu.mx.RLocker()
614+
rlock.Lock()
615+
defer rlock.Unlock()
613616
stats, ok = s.mu.txns[key]
614617
return ok
615618
}()
616619
if ok || !createIfNonexistent {
617620
return stats, false /* created */, false /* throttled */
618621
}
619-
620622
// Key does not exist in map. Take a full lock to add the key.
621-
s.mu.Lock()
622-
defer s.mu.Unlock()
623+
s.mu.mx.Lock()
624+
defer s.mu.mx.Unlock()
625+
623626
return s.getStatsForTxnWithKeyLocked(key, stmtFingerprintIDs, createIfNonexistent)
624627
}
625628

@@ -657,8 +660,9 @@ func (s *Container) getStatsForTxnWithKeyLocked(
657660

658661
// SaveToLog saves the existing statement stats into the info log.
659662
func (s *Container) SaveToLog(ctx context.Context, appName string) {
660-
s.mu.RLock()
661-
defer s.mu.RUnlock()
663+
rlock := s.mu.mx.RLocker()
664+
rlock.Lock()
665+
defer rlock.Unlock()
662666
if len(s.mu.stmts) == 0 {
663667
return
664668
}
@@ -679,36 +683,28 @@ func (s *Container) SaveToLog(ctx context.Context, appName string) {
679683
// Clear clears the data stored in this Container and prepare the Container
680684
// for reuse.
681685
func (s *Container) Clear(ctx context.Context) {
682-
s.mu.Lock()
683-
defer s.mu.Unlock()
684-
685-
s.freeLocked(ctx)
686-
686+
s.Free(ctx)
687+
688+
s.mu.mx.Lock()
689+
defer s.mu.mx.Unlock()
687690
// Clear the map, to release the memory; make the new map somewhat already
688691
// large for the likely future workload.
689692
s.mu.stmts = make(map[stmtKey]*stmtStats, len(s.mu.stmts)/2)
690693
s.mu.txns = make(map[appstatspb.TransactionFingerprintID]*txnStats, len(s.mu.txns)/2)
691694

692-
s.muPlanCache.Lock()
693-
defer s.muPlanCache.Unlock()
694-
s.muPlanCache.sampledPlanMetadataCache = make(map[sampledPlanKey]time.Time, len(s.muPlanCache.sampledPlanMetadataCache)/2)
695+
s.mu.sampledPlanMetadataCache = make(map[sampledPlanKey]time.Time, len(s.mu.sampledPlanMetadataCache)/2)
695696
}
696697

697698
// Free frees the accounted resources from the Container. The Container is
698699
// presumed to be no longer in use and its actual allocated memory will
699700
// eventually be GC'd.
700701
func (s *Container) Free(ctx context.Context) {
701-
s.mu.Lock()
702-
defer s.mu.Unlock()
703-
704-
s.freeLocked(ctx)
705-
}
706-
707-
func (s *Container) freeLocked(ctx context.Context) {
708702
atomic.AddInt64(s.atomic.uniqueStmtFingerprintCount, int64(-len(s.mu.stmts)))
709-
atomic.AddInt64(s.atomic.uniqueTxnFingerprintCount, int64(-len(s.mu.txns)))
703+
atomic.AddInt64(s.atomic.uniqueTxnFingerprintCount, int64(-len(s.mu.txns)))
710704

711-
s.mu.acc.Clear(ctx)
705+
s.muAcc.Lock()
706+
s.muAcc.acc.Clear(ctx)
707+
s.muAcc.Unlock()
712708
}
713709

714710
// MergeApplicationStatementStats implements the sqlstats.ApplicationStats interface.
@@ -801,8 +797,9 @@ func (s *Container) MergeApplicationTransactionStats(
801797
// a lock on a will cause a deadlock.
802798
func (s *Container) Add(ctx context.Context, other *Container) (err error) {
803799
statMap := func() map[stmtKey]*stmtStats {
804-
other.mu.RLock()
805-
defer other.mu.RUnlock()
800+
rlock := other.mu.mx.RLocker()
801+
rlock.Lock()
802+
defer rlock.Unlock()
806803

807804
statMap := make(map[stmtKey]*stmtStats)
808805
for k, v := range other.mu.stmts {
@@ -845,11 +842,14 @@ func (s *Container) Add(ctx context.Context, other *Container) (err error) {
845842
// We still want to continue this loop to merge stats that are already
846843
// present in our map that do not require allocation.
847844
if latestErr := func() error {
848-
s.mu.Lock()
849-
defer s.mu.Unlock()
850-
growErr := s.mu.acc.Grow(ctx, estimatedAllocBytes)
845+
s.muAcc.Lock()
846+
growErr := s.muAcc.acc.Grow(ctx, estimatedAllocBytes)
847+
s.muAcc.Unlock()
848+
851849
if growErr != nil {
850+
s.mu.mx.Lock()
852851
delete(s.mu.stmts, k)
852+
s.mu.mx.Unlock()
853853
}
854854
return growErr
855855
}(); latestErr != nil {
@@ -871,8 +871,8 @@ func (s *Container) Add(ctx context.Context, other *Container) (err error) {
871871

872872
// Do what we did above for the statMap for the txn Map now.
873873
txnMap := func() map[appstatspb.TransactionFingerprintID]*txnStats {
874-
other.mu.Lock()
875-
defer other.mu.Unlock()
874+
other.mu.mx.Lock()
875+
defer other.mu.mx.Unlock()
876876
txnMap := make(map[appstatspb.TransactionFingerprintID]*txnStats)
877877
for k, v := range other.mu.txns {
878878
txnMap[k] = v
@@ -915,10 +915,10 @@ func (s *Container) Add(ctx context.Context, other *Container) (err error) {
915915
// We still want to continue this loop to merge stats that are already
916916
// present in our map that do not require allocation.
917917
if latestErr := func() error {
918-
s.mu.Lock()
919-
defer s.mu.Unlock()
918+
s.muAcc.Lock()
919+
growErr := s.muAcc.acc.Grow(ctx, estimatedAllocBytes)
920+
s.muAcc.Unlock()
920921

921-
growErr := s.mu.acc.Grow(ctx, estimatedAllocBytes)
922922
if growErr != nil {
923923
delete(s.mu.txns, k)
924924
}
@@ -979,16 +979,17 @@ func (s *transactionCounts) recordTransactionCounts(
979979
func (s *Container) getLogicalPlanLastSampled(
980980
key sampledPlanKey,
981981
) (lastSampled time.Time, found bool) {
982-
s.muPlanCache.RLock()
983-
defer s.muPlanCache.RUnlock()
984-
lastSampled, found = s.muPlanCache.sampledPlanMetadataCache[key]
982+
rlock := s.mu.mx.RLocker()
983+
rlock.Lock()
984+
defer rlock.Unlock()
985+
lastSampled, found = s.mu.sampledPlanMetadataCache[key]
985986
return lastSampled, found
986987
}
987988

988989
func (s *Container) setLogicalPlanLastSampled(key sampledPlanKey, time time.Time) {
989-
s.muPlanCache.Lock()
990-
defer s.muPlanCache.Unlock()
991-
s.muPlanCache.sampledPlanMetadataCache[key] = time
990+
s.mu.mx.Lock()
991+
defer s.mu.mx.Unlock()
992+
s.mu.sampledPlanMetadataCache[key] = time
992993
}
993994

994995
// shouldSaveLogicalPlanDescription returns whether we should save the sample

0 commit comments

Comments
 (0)