Skip to content

Commit 2a307b2

Browse files
craig[bot]yuzefovichstevendanna
committed
149053: sql: fix cancel leak in happy case in fingerprint_span r=yuzefovich a=yuzefovich The recent fix introduced a possible leak of the cancel function (namely, in the happy case when the coordinator doesn't hit an early error, we would never call the cancellation function). We should just defer it unconditionally and might call it earlier on a shutdown due to an error. Epic: None Release note: None 149197: kvnemesis: log seed and rng access count r=miraradeva a=stevendanna This new logging 1) Ensures the seed is captured as part of the kvnemesis output even if the test runner logs are somehow lost. 2) Allows us to spot when a seed might not be reproducing a problem because of a change in the number of random choices in KVNemesis. Epic: none Release note: None Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: Steven Danna <[email protected]>
3 parents 9f45451 + bdb0d7a + 2dce023 commit 2a307b2

File tree

5 files changed

+75
-11
lines changed

5 files changed

+75
-11
lines changed

pkg/kv/kvnemesis/generator.go

Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ import (
1414
"math/rand"
1515
"slices"
1616
"sort"
17+
"sync/atomic"
1718

1819
"github.com/cockroachdb/cockroach/pkg/keys"
1920
"github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil"
@@ -36,6 +37,9 @@ type GeneratorConfig struct {
3637
NumNodes, NumReplicas int
3738

3839
BufferedWritesProb float64
40+
41+
SeedForLogging int64
42+
RandSourceCounterForLogging counter
3943
}
4044

4145
// OperationConfig configures the relative probabilities of producing various
@@ -1993,3 +1997,40 @@ func releaseSavepoint(id int) Operation {
19931997
func rollbackSavepoint(id int) Operation {
19941998
return Operation{SavepointRollback: &SavepointRollbackOperation{ID: int32(id)}}
19951999
}
2000+
2001+
type countingRandSource struct {
2002+
count atomic.Uint64
2003+
inner rand.Source64
2004+
}
2005+
2006+
type counter interface {
2007+
Count() uint64
2008+
}
2009+
2010+
// newCountingSource creates random source that counts how many times it was
2011+
// called for logging purposes.
2012+
func newCountingSource(inner rand.Source64) *countingRandSource {
2013+
return &countingRandSource{
2014+
inner: inner,
2015+
}
2016+
}
2017+
2018+
func (c *countingRandSource) Count() uint64 {
2019+
return c.count.Load()
2020+
}
2021+
2022+
func (c *countingRandSource) Int63() int64 {
2023+
c.count.Add(1)
2024+
return c.inner.Int63()
2025+
}
2026+
2027+
func (c *countingRandSource) Uint64() uint64 {
2028+
c.count.Add(1)
2029+
return c.inner.Uint64()
2030+
}
2031+
2032+
func (c *countingRandSource) Seed(seed int64) {
2033+
// We assume that seed invalidates the count.
2034+
c.count.Store(0)
2035+
c.inner.Seed(seed)
2036+
}

pkg/kv/kvnemesis/kvnemesis.go

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,7 +177,13 @@ func RunNemesis(
177177
failuresFile = l(ctx, "failures", "%s", &buf)
178178
}
179179

180-
reproFile := l(ctx, "repro.go", "// Reproduction steps:\n%s", printRepro(stepsByWorker))
180+
reproFile := l(ctx, "repro.go", `// Seed: %d
181+
// Calls to Random Source: %d
182+
// Reproduction steps:
183+
%s`,
184+
config.SeedForLogging,
185+
config.RandSourceCounterForLogging.Count(),
186+
printRepro(stepsByWorker))
181187
rangefeedFile := l(ctx, "kvs-rangefeed.txt", "kvs (recorded from rangefeed):\n%s", kvs.DebugPrint(" "))
182188
kvsFile := "<error>"
183189
var scanKVs []kv.KeyValue

pkg/kv/kvnemesis/kvnemesis_test.go

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -210,16 +210,17 @@ func randWithSeed(
210210
Logf(string, ...interface{})
211211
Helper()
212212
}, seedOrZero int64,
213-
) *rand.Rand {
213+
) (*rand.Rand, counter, int64) {
214214
t.Helper()
215-
var rng *rand.Rand
215+
var rngSource rand.Source
216216
if seedOrZero > 0 {
217-
rng = rand.New(rand.NewSource(seedOrZero))
217+
rngSource = rand.NewSource(seedOrZero)
218218
} else {
219-
rng, seedOrZero = randutil.NewTestRand()
219+
rngSource, seedOrZero = randutil.NewTestRandSource()
220220
}
221221
t.Logf("seed: %d", seedOrZero)
222-
return rng
222+
countingSource := newCountingSource(rngSource.(rand.Source64))
223+
return rand.New(countingSource), countingSource, seedOrZero
223224
}
224225

225226
type ti interface {
@@ -445,7 +446,7 @@ func testKVNemesisImpl(t *testing.T, cfg kvnemesisTestCfg) {
445446

446447
// Can set a seed here for determinism. This works best when the seed was
447448
// obtained with cfg.concurrency=1.
448-
rng := randWithSeed(t, cfg.seedOverride)
449+
rng, countingSource, seed := randWithSeed(t, cfg.seedOverride)
449450

450451
// 4 nodes so we have somewhere to move 3x replicated ranges to.
451452
ctx := context.Background()
@@ -466,6 +467,10 @@ func testKVNemesisImpl(t *testing.T, cfg kvnemesisTestCfg) {
466467
config.NumNodes = cfg.numNodes
467468
config.NumReplicas = 3
468469
config.BufferedWritesProb = cfg.bufferedWriteProb
470+
471+
config.SeedForLogging = seed
472+
config.RandSourceCounterForLogging = countingSource
473+
469474
if config.NumReplicas > cfg.numNodes {
470475
config.NumReplicas = cfg.numNodes
471476
}

pkg/sql/fingerprint_span.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -139,6 +139,7 @@ func (p *planner) fingerprintSpanFanout(
139139
// the coordinator encounters an error.
140140
workCh := make(chan roachpb.Span)
141141
ctx, cancel := context.WithCancel(ctx)
142+
defer cancel()
142143

143144
grp := ctxgroup.WithContext(ctx)
144145
for range maxWorkerCount {

pkg/util/randutil/rand.go

Lines changed: 15 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -115,16 +115,27 @@ func NewPseudoRandWithGlobalSeed() (*rand.Rand, int64) {
115115
// seed. This rand.Rand is useful in testing to produce deterministic,
116116
// reproducible behavior.
117117
func NewTestRand() (*rand.Rand, int64) {
118-
return newTestRandImpl(rand.NewSource)
118+
src, seed := newTestRandSourceImpl(rand.NewSource)
119+
return rand.New(src), seed
120+
}
121+
122+
// NewTestRandSource returns a math/rand.Source64 seeded from rng, which is seeded
123+
// with the global seed. If the caller is a test with a different path-qualified
124+
// name than the previous caller, rng is reseeded from the global seed. This
125+
// random source is useful in testing to produce deterministic, reproducible
126+
// behavior.
127+
func NewTestRandSource() (rand.Source, int64) {
128+
return newTestRandSourceImpl(rand.NewSource)
119129
}
120130

121131
// NewLockedTestRand is identical to NewTestRand but returned rand.Rand is using
122132
// thread safe underlying source.
123133
func NewLockedTestRand() (*rand.Rand, int64) {
124-
return newTestRandImpl(NewLockedSource)
134+
src, seed := newTestRandSourceImpl(NewLockedSource)
135+
return rand.New(src), seed
125136
}
126137

127-
func newTestRandImpl(f func(int64) rand.Source) (*rand.Rand, int64) {
138+
func newTestRandSourceImpl(f func(int64) rand.Source) (rand.Source, int64) {
128139
mtx.Lock()
129140
defer mtx.Unlock()
130141
fxn := getTestName()
@@ -136,7 +147,7 @@ func newTestRandImpl(f func(int64) rand.Source) (*rand.Rand, int64) {
136147
rng = rand.New(f(globalSeed))
137148
}
138149
seed := rng.Int63()
139-
return rand.New(f(seed)), seed
150+
return f(seed), seed
140151
}
141152

142153
// NewTestRandWithSeed returns an instance of math/rand.Rand, similar to

0 commit comments

Comments
 (0)