Skip to content

Commit 8a8e228

Browse files
committed
*: replace some usages of global rand
When we use `rand.*` methods like `math/rand.Intn`, under the hood we hit the global rand source, which is protected by a mutex, so we could be subject to contention on that lock. This commit updates some of such usages in favor of object-specific rand source to avoid that. Note that allocating a separate rand source is not free (the allocation itself is about 5KiB in size), so we modify spots where the lifecycle matches that of the server or when the affected code is heavy already / not on the hot path (also when it's clear that the access is from within a single goroutine). Release note: None
1 parent 6415f16 commit 8a8e228

File tree

11 files changed

+34
-19
lines changed

11 files changed

+34
-19
lines changed

pkg/gossip/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ go_library(
3434
"//pkg/util/log",
3535
"//pkg/util/metric",
3636
"//pkg/util/protoutil",
37+
"//pkg/util/randutil",
3738
"//pkg/util/stop",
3839
"//pkg/util/syncutil",
3940
"//pkg/util/timeutil",

pkg/gossip/gossip.go

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -62,6 +62,7 @@ import (
6262
"github.com/cockroachdb/cockroach/pkg/util/log"
6363
"github.com/cockroachdb/cockroach/pkg/util/metric"
6464
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
65+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
6566
"github.com/cockroachdb/cockroach/pkg/util/stop"
6667
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
6768
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -1360,9 +1361,10 @@ func (g *Gossip) manage(rpcContext *rpc.Context) {
13601361
var cullTimer, stallTimer timeutil.Timer
13611362
defer cullTimer.Stop()
13621363
defer stallTimer.Stop()
1364+
rng, _ := randutil.NewPseudoRand()
13631365

1364-
cullTimer.Reset(jitteredInterval(g.cullInterval))
1365-
stallTimer.Reset(jitteredInterval(g.stallInterval))
1366+
cullTimer.Reset(jitteredInterval(g.cullInterval, rng))
1367+
stallTimer.Reset(jitteredInterval(g.stallInterval, rng))
13661368
for {
13671369
select {
13681370
case <-g.server.stopper.ShouldQuiesce():
@@ -1372,7 +1374,7 @@ func (g *Gossip) manage(rpcContext *rpc.Context) {
13721374
case <-g.tighten:
13731375
g.tightenNetwork(ctx, rpcContext)
13741376
case <-cullTimer.C:
1375-
cullTimer.Reset(jitteredInterval(g.cullInterval))
1377+
cullTimer.Reset(jitteredInterval(g.cullInterval, rng))
13761378
func() {
13771379
g.mu.Lock()
13781380
if !g.outgoing.hasSpace() {
@@ -1402,7 +1404,7 @@ func (g *Gossip) manage(rpcContext *rpc.Context) {
14021404
g.mu.Unlock()
14031405
}()
14041406
case <-stallTimer.C:
1405-
stallTimer.Reset(jitteredInterval(g.stallInterval))
1407+
stallTimer.Reset(jitteredInterval(g.stallInterval, rng))
14061408
func() {
14071409
g.mu.Lock()
14081410
defer g.mu.Unlock()
@@ -1415,8 +1417,8 @@ func (g *Gossip) manage(rpcContext *rpc.Context) {
14151417

14161418
// jitteredInterval returns a randomly jittered (+/-25%) duration
14171419
// from checkInterval.
1418-
func jitteredInterval(interval time.Duration) time.Duration {
1419-
return time.Duration(float64(interval) * (0.75 + 0.5*rand.Float64()))
1420+
func jitteredInterval(interval time.Duration, rng *rand.Rand) time.Duration {
1421+
return time.Duration(float64(interval) * (0.75 + 0.5*rng.Float64()))
14201422
}
14211423

14221424
// tightenNetwork "tightens" the network by starting a new gossip client to the

pkg/server/diagnostics/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@ go_library(
3636
"//pkg/util/log",
3737
"//pkg/util/log/logcrash",
3838
"//pkg/util/protoutil",
39+
"//pkg/util/randutil",
3940
"//pkg/util/stop",
4041
"//pkg/util/syncutil",
4142
"//pkg/util/system",

pkg/server/diagnostics/diagnostics.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -149,9 +149,9 @@ func addInfoToURL(
149149
}
150150

151151
// randomly shift `d` to be up to `jitterSeconds` shorter or longer.
152-
func addJitter(d time.Duration) time.Duration {
152+
func addJitter(d time.Duration, rng *rand.Rand) time.Duration {
153153
const jitterSeconds = 120
154-
j := time.Duration(rand.Intn(jitterSeconds*2)-jitterSeconds) * time.Second
154+
j := time.Duration(rng.Intn(jitterSeconds*2)-jitterSeconds) * time.Second
155155
return d + j
156156
}
157157

pkg/server/diagnostics/reporter.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ import (
4040
"github.com/cockroachdb/cockroach/pkg/util/log"
4141
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
4242
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
43+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
4344
"github.com/cockroachdb/cockroach/pkg/util/stop"
4445
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
4546
"github.com/cockroachdb/cockroach/pkg/util/uuid"
@@ -185,6 +186,7 @@ func (r *Reporter) PeriodicallyReportDiagnostics(ctx context.Context, stopper *s
185186
ctx, cancel = stopper.WithCancelOnQuiesce(ctx)
186187
defer cancel()
187188
defer logcrash.RecoverAndReportNonfatalPanic(ctx, &r.Settings.SV)
189+
rng, _ := randutil.NewPseudoRand()
188190
nextReport := r.StartTime
189191

190192
var timer timeutil.Timer
@@ -199,7 +201,7 @@ func (r *Reporter) PeriodicallyReportDiagnostics(ctx context.Context, stopper *s
199201

200202
nextReport = nextReport.Add(reportFrequency.Get(&r.Settings.SV))
201203

202-
timer.Reset(addJitter(nextReport.Sub(timeutil.Now())))
204+
timer.Reset(addJitter(nextReport.Sub(timeutil.Now()), rng))
203205
select {
204206
case <-stopper.ShouldQuiesce():
205207
return

pkg/server/diagnostics/update_checker.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import (
2222
"github.com/cockroachdb/cockroach/pkg/util/httputil"
2323
"github.com/cockroachdb/cockroach/pkg/util/log"
2424
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
25+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
2526
"github.com/cockroachdb/cockroach/pkg/util/stop"
2627
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2728
"github.com/cockroachdb/cockroach/pkg/util/uuid"
@@ -74,6 +75,7 @@ func (u *UpdateChecker) PeriodicallyCheckForUpdates(ctx context.Context, stopper
7475
SpanOpt: stop.SterileRootSpan,
7576
}, func(ctx context.Context) {
7677
defer logcrash.RecoverAndReportNonfatalPanic(ctx, &u.Settings.SV)
78+
rng, _ := randutil.NewPseudoRand()
7779
nextUpdateCheck := u.StartTime
7880

7981
var timer timeutil.Timer
@@ -84,7 +86,7 @@ func (u *UpdateChecker) PeriodicallyCheckForUpdates(ctx context.Context, stopper
8486

8587
nextUpdateCheck = u.maybeCheckForUpdates(ctx, now, nextUpdateCheck, runningTime)
8688

87-
timer.Reset(addJitter(nextUpdateCheck.Sub(timeutil.Now())))
89+
timer.Reset(addJitter(nextUpdateCheck.Sub(timeutil.Now()), rng))
8890
select {
8991
case <-stopper.ShouldQuiesce():
9092
return

pkg/sql/catalog/lease/storage.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,9 @@ var LeaseRenewalCrossValidate = settings.RegisterBoolSetting(
111111
func (s storage) jitteredLeaseDuration() time.Duration {
112112
leaseDuration := LeaseDuration.Get(&s.settings.SV)
113113
jitterFraction := LeaseJitterFraction.Get(&s.settings.SV)
114+
// TODO(yuzefovich): it would probably be worth replacing this usage of
115+
// global rand with rng tied to the 'storage' object. It's not clear whether
116+
// we need concurrency safety or not.
114117
return time.Duration(float64(leaseDuration) * (1 - jitterFraction +
115118
2*jitterFraction*rand.Float64()))
116119
}

pkg/sql/contention/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ go_library(
2929
"//pkg/util/log/eventpb",
3030
"//pkg/util/log/logpb",
3131
"//pkg/util/metric",
32+
"//pkg/util/randutil",
3233
"//pkg/util/stop",
3334
"//pkg/util/syncutil",
3435
"//pkg/util/timeutil",

pkg/sql/contention/event_store.go

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121
"github.com/cockroachdb/cockroach/pkg/util/log"
2222
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
2323
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
24+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
2425
"github.com/cockroachdb/cockroach/pkg/util/stop"
2526
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
2627
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -206,15 +207,15 @@ func (s *eventStore) startResolver(ctx context.Context, stopper *stop.Stopper) {
206207
})
207208

208209
_ = stopper.RunAsyncTask(ctx, "contention-event-resolver", func(ctx context.Context) {
209-
210-
initialDelay := s.resolutionIntervalWithJitter()
210+
rng, _ := randutil.NewPseudoRand()
211+
initialDelay := s.resolutionIntervalWithJitter(rng)
211212
var timer timeutil.Timer
212213
defer timer.Stop()
213214

214215
timer.Reset(initialDelay)
215216

216217
for {
217-
waitInterval := s.resolutionIntervalWithJitter()
218+
waitInterval := s.resolutionIntervalWithJitter(rng)
218219
timer.Reset(waitInterval)
219220

220221
select {
@@ -363,11 +364,11 @@ func (s *eventStore) addEventsForTest(events []contentionpb.ExtendedContentionEv
363364
}
364365
}
365366

366-
func (s *eventStore) resolutionIntervalWithJitter() time.Duration {
367+
func (s *eventStore) resolutionIntervalWithJitter(rng *rand.Rand) time.Duration {
367368
baseInterval := TxnIDResolutionInterval.Get(&s.st.SV)
368369

369370
// Jitter the interval a by +/- 15%.
370-
frac := 1 + (2*rand.Float64()-1)*0.15
371+
frac := 1 + (2*rng.Float64()-1)*0.15
371372
jitteredInterval := time.Duration(frac * float64(baseInterval.Nanoseconds()))
372373
return jitteredInterval
373374
}

pkg/sql/index_split_scatter.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ package sql
88
import (
99
"bytes"
1010
"context"
11-
"math/rand"
1211
"sort"
1312
"time"
1413

@@ -27,6 +26,7 @@ import (
2726
"github.com/cockroachdb/cockroach/pkg/sql/stats"
2827
"github.com/cockroachdb/cockroach/pkg/util/encoding"
2928
"github.com/cockroachdb/cockroach/pkg/util/log"
29+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
3030
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
3131
"github.com/cockroachdb/errors"
3232
)
@@ -317,6 +317,7 @@ func (is *indexSplitAndScatter) MaybeSplitIndexSpans(
317317
if is.testingKnobs.BeforeIndexSplitAndScatter != nil {
318318
is.testingKnobs.BeforeIndexSplitAndScatter(splitPoints)
319319
}
320+
rng, _ := randutil.NewPseudoRand()
320321
for i := 0; i < nSplits; i++ {
321322
// Evenly space out the ranges that we select from the ranges that are
322323
// returned.
@@ -328,7 +329,7 @@ func (is *indexSplitAndScatter) MaybeSplitIndexSpans(
328329

329330
// Jitter the expiration time by 20% up or down from the default.
330331
maxJitter := backfillSplitExpiration.Nanoseconds() / 5
331-
jitter := rand.Int63n(maxJitter*2) - maxJitter
332+
jitter := rng.Int63n(maxJitter*2) - maxJitter
332333
expirationTime := backfillSplitExpiration.Nanoseconds() + jitter
333334

334335
b.AddRawRequest(&kvpb.AdminSplitRequest{

0 commit comments

Comments
 (0)