Skip to content

Commit dc28e19

Browse files
craig[bot]Renato Costafqazinvb
committed
107756: roachtest: make path to default cockroach public r=srosenberg a=renatolabs The roachtest test runner already uploads the `cockroach` binary passed with `--cockroach` to every node in the cluster. By making the path to that file public, we allow tests to use that binary, stopping avoidable uploads. Concretely, this commit removes a step from `mixedversion` tests where the same logic of uploading the current binary took place. Tests can use the new constant in the `test` package instead. Epic: none Release note: None 108082: sql: remove slow functions from TestRandomSyntaxFunctions r=fqazi a=fqazi Previously, we could easily timeout running slower, functions in the random syntax functions test. We attempted to minimize this risk with resettable timeouts which helped, but libpq has limited support for cancellation, so we need to fully pull these out. To address this, this patch will remove: crdb_internal.revalidate_unique_constraints_in_all_tables and crdb_internal.validate_ttl_scheduled_jobs from testing. Fixes: cockroachdb#107929 Release note: None 108118: kv: deflake TestFollowerReadsWithStaleDescriptor r=nvanbenschoten a=nvanbenschoten Fixes cockroachdb#108087. This fix avoids a data race in the test. The race was harmless, but could cause the test to fail when run with the race detector. Release note: None Co-authored-by: Renato Costa <[email protected]> Co-authored-by: Faizan Qazi <[email protected]> Co-authored-by: Nathan VanBenschoten <[email protected]>
4 parents 76b5348 + 97dba4f + 7a97af5 + 6db50f7 commit dc28e19

File tree

8 files changed

+66
-86
lines changed

8 files changed

+66
-86
lines changed

pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ import (
1414
"fmt"
1515
"math"
1616
"strings"
17+
"sync/atomic"
1718
"testing"
1819
"time"
1920

@@ -697,7 +698,8 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
697698
// The test uses follower_read_timestamp().
698699
defer utilccl.TestingEnableEnterprise()()
699700

700-
historicalQuery := `SELECT * FROM test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2`
701+
var historicalQuery atomic.Value
702+
historicalQuery.Store(`SELECT * FROM test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2`)
701703
recCh := make(chan tracingpb.Recording, 1)
702704

703705
tc := testcluster.StartTestCluster(t, 4,
@@ -732,7 +734,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
732734
},
733735
SQLExecutor: &sql.ExecutorTestingKnobs{
734736
WithStatementTrace: func(trace tracingpb.Recording, stmt string) {
735-
if stmt == historicalQuery {
737+
if stmt == historicalQuery.Load().(string) {
736738
recCh <- trace
737739
}
738740
},
@@ -786,7 +788,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
786788
// not be executed as a follower read since it attempts to use n2 which
787789
// doesn't have a replica any more and then it tries n1 which returns an
788790
// updated descriptor.
789-
n4.Exec(t, historicalQuery)
791+
n4.Exec(t, historicalQuery.Load().(string))
790792
// As a sanity check, verify that this was not a follower read.
791793
rec := <-recCh
792794
require.False(t, kv.OnlyFollowerReads(rec), "query was served through follower reads: %s", rec)
@@ -812,7 +814,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
812814
// Run a historical query and assert that it's served from the follower (n3).
813815
// n4 should attempt to route to n3 because we pretend n3 has a lower latency
814816
// (see testing knob).
815-
n4.Exec(t, historicalQuery)
817+
n4.Exec(t, historicalQuery.Load().(string))
816818
rec = <-recCh
817819

818820
// Look at the trace and check that we've served a follower read.
@@ -855,8 +857,8 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
855857
// the ReplicaInfo twice for the same range. This allows us to verify that
856858
// the cached - in the spanResolverIterator - information is correctly
857859
// preserved.
858-
historicalQuery = `SELECT * FROM [SELECT * FROM test WHERE k=2 UNION ALL SELECT * FROM test WHERE k=3] AS OF SYSTEM TIME follower_read_timestamp()`
859-
n4.Exec(t, historicalQuery)
860+
historicalQuery.Store(`SELECT * FROM [SELECT * FROM test WHERE k=2 UNION ALL SELECT * FROM test WHERE k=3] AS OF SYSTEM TIME follower_read_timestamp()`)
861+
n4.Exec(t, historicalQuery.Load().(string))
860862
rec = <-recCh
861863

862864
// Sanity check that the plan was distributed.

pkg/cmd/roachtest/cluster.go

Lines changed: 6 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,6 @@ const (
115115
defaultEncryptionProbability = 1
116116
defaultFIPSProbability = 0
117117
defaultARM64Probability = 0
118-
defaultCockroachPath = "./cockroach-default"
119118
)
120119

121120
type errBinaryOrLibraryNotFound struct {
@@ -1347,7 +1346,7 @@ func (c *clusterImpl) FetchLogs(ctx context.Context, l *logger.Logger) error {
13471346
}
13481347
}
13491348

1350-
if err := c.RunE(ctx, c.All(), fmt.Sprintf("mkdir -p logs/redacted && %s debug merge-logs --redact logs/*.log > logs/redacted/combined.log", defaultCockroachPath)); err != nil {
1349+
if err := c.RunE(ctx, c.All(), fmt.Sprintf("mkdir -p logs/redacted && %s debug merge-logs --redact logs/*.log > logs/redacted/combined.log", test.DefaultCockroachPath)); err != nil {
13511350
l.Printf("failed to redact logs: %v", err)
13521351
if ctx.Err() != nil {
13531352
return err
@@ -1428,7 +1427,7 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger)
14281427
sec = fmt.Sprintf("--certs-dir=%s", certs)
14291428
}
14301429
if err := c.RunE(
1431-
ctx, c.Node(node), fmt.Sprintf("%s debug tsdump %s --format=raw > tsdump.gob", defaultCockroachPath, sec),
1430+
ctx, c.Node(node), fmt.Sprintf("%s debug tsdump %s --format=raw > tsdump.gob", test.DefaultCockroachPath, sec),
14321431
); err != nil {
14331432
return err
14341433
}
@@ -1499,15 +1498,15 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger, dest
14991498
// Ignore the files in the the log directory; we pull the logs separately anyway
15001499
// so this would only cause duplication.
15011500
excludeFiles := "*.log,*.txt,*.pprof"
1502-
cmd := roachtestutil.NewCommand("%s debug zip", defaultCockroachPath).
1501+
cmd := roachtestutil.NewCommand("%s debug zip", test.DefaultCockroachPath).
15031502
Option("include-range-info").
15041503
Flag("exclude-files", fmt.Sprintf("'%s'", excludeFiles)).
15051504
Flag("url", fmt.Sprintf("{pgurl:%d}", i)).
15061505
MaybeFlag(c.IsSecure(), "certs-dir", "certs").
15071506
Arg(zipName).
15081507
String()
15091508
if err := c.RunE(ctx, c.Node(i), cmd); err != nil {
1510-
l.Printf("%s debug zip failed on node %d: %v", defaultCockroachPath, i, err)
1509+
l.Printf("%s debug zip failed on node %d: %v", test.DefaultCockroachPath, i, err)
15111510
if i < c.spec.NodeCount {
15121511
continue
15131512
}
@@ -1928,15 +1927,15 @@ func (c *clusterImpl) PutE(
19281927
}
19291928

19301929
// PutDefaultCockroach uploads the cockroach binary passed in the
1931-
// command line to `defaultCockroachPath` in every node in the
1930+
// command line to `test.DefaultCockroachPath` in every node in the
19321931
// cluster. This binary is used by the test runner to collect failure
19331932
// artifacts since tests are free to upload the cockroach binary they
19341933
// use to any location they desire.
19351934
func (c *clusterImpl) PutDefaultCockroach(
19361935
ctx context.Context, l *logger.Logger, cockroachPath string,
19371936
) error {
19381937
c.status("uploading default cockroach binary to nodes")
1939-
return c.PutE(ctx, l, cockroachPath, defaultCockroachPath, c.All())
1938+
return c.PutE(ctx, l, cockroachPath, test.DefaultCockroachPath, c.All())
19401939
}
19411940

19421941
// PutLibraries inserts the specified libraries, by name, into all nodes on the cluster

pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go

Lines changed: 0 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -111,13 +111,6 @@ const (
111111
// cluster that can use the test fixtures in
112112
// `pkg/cmd/roachtest/fixtures`.
113113
numNodesInFixtures = 4
114-
115-
// CurrentCockroachPath is the path to the binary where the current
116-
// version of cockroach being tested is located. This file is
117-
// uploaded before any user functions are run. The primary use case
118-
// are tests that need long runnig background functions on startup
119-
// (such as running a workload).
120-
CurrentCockroachPath = "./cockroach-current"
121114
)
122115

123116
var (
@@ -581,35 +574,6 @@ func (s startStep) Run(ctx context.Context, l *logger.Logger, c cluster.Cluster,
581574
return clusterupgrade.StartWithSettings(ctx, l, c, s.crdbNodes, startOpts, clusterSettings...)
582575
}
583576

584-
// uploadCurrentVersionStep uploads the current cockroach binary to
585-
// all DB nodes in the test. This is so that startup steps can use
586-
// them (if, for instance, they need to run a workload). The binary
587-
// will be located in `dest`.
588-
type uploadCurrentVersionStep struct {
589-
id int
590-
rt test.Test
591-
crdbNodes option.NodeListOption
592-
dest string
593-
}
594-
595-
func (s uploadCurrentVersionStep) ID() int { return s.id }
596-
func (s uploadCurrentVersionStep) Background() shouldStop { return nil }
597-
598-
func (s uploadCurrentVersionStep) Description() string {
599-
return fmt.Sprintf("upload current binary to all cockroach nodes (%v)", s.crdbNodes)
600-
}
601-
602-
func (s uploadCurrentVersionStep) Run(
603-
ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper,
604-
) error {
605-
_, err := clusterupgrade.UploadVersion(ctx, s.rt, l, c, s.crdbNodes, clusterupgrade.MainVersion)
606-
if err != nil {
607-
return err
608-
}
609-
610-
return c.RunE(ctx, s.crdbNodes, fmt.Sprintf("mv ./cockroach %s", s.dest))
611-
}
612-
613577
// waitForStableClusterVersionStep implements the process of waiting
614578
// for the `version` cluster setting being the same on all nodes of
615579
// the cluster and equal to the binary version of the first node in

pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -132,7 +132,6 @@ func (p *testPlanner) initSteps() []testStep {
132132

133133
return append(
134134
append(steps,
135-
uploadCurrentVersionStep{id: p.nextID(), rt: p.rt, crdbNodes: p.crdbNodes, dest: CurrentCockroachPath},
136135
waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes, timeout: p.options.upgradeTimeout},
137136
preserveDowngradeOptionStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes},
138137
),

pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go

Lines changed: 33 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -69,45 +69,44 @@ func TestTestPlanner(t *testing.T) {
6969

7070
plan, err := mvt.plan()
7171
require.NoError(t, err)
72-
require.Len(t, plan.steps, 11)
72+
require.Len(t, plan.steps, 10)
7373

7474
// Assert on the pretty-printed version of the test plan as that
7575
// asserts the ordering of the steps we want to take, and as a bonus
7676
// tests the printing function itself.
7777
expectedPrettyPlan := fmt.Sprintf(`
7878
mixed-version test plan for upgrading from %[1]s to <current>:
7979
├── starting cluster at version "%[1]s" (1)
80-
├── upload current binary to all cockroach nodes (:1-4) (2)
81-
├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (3)
82-
├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (4)
83-
├── run "initialize bank workload" (5)
80+
├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (2)
81+
├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (3)
82+
├── run "initialize bank workload" (4)
8483
├── start background hooks concurrently
85-
│ ├── run "bank workload", after 50ms delay (6)
86-
│ ├── run "rand workload", after 200ms delay (7)
87-
│ └── run "csv server", after 500ms delay (8)
84+
│ ├── run "bank workload", after 50ms delay (5)
85+
│ ├── run "rand workload", after 200ms delay (6)
86+
│ └── run "csv server", after 500ms delay (7)
8887
├── upgrade nodes :1-4 from "%[1]s" to "<current>"
89-
│ ├── restart node 1 with binary version <current> (9)
90-
│ ├── run "mixed-version 1" (10)
91-
│ ├── restart node 4 with binary version <current> (11)
92-
│ ├── restart node 3 with binary version <current> (12)
93-
│ ├── run "mixed-version 2" (13)
94-
│ └── restart node 2 with binary version <current> (14)
88+
│ ├── restart node 1 with binary version <current> (8)
89+
│ ├── run "mixed-version 1" (9)
90+
│ ├── restart node 4 with binary version <current> (10)
91+
│ ├── restart node 3 with binary version <current> (11)
92+
│ ├── run "mixed-version 2" (12)
93+
│ └── restart node 2 with binary version <current> (13)
9594
├── downgrade nodes :1-4 from "<current>" to "%[1]s"
96-
│ ├── restart node 4 with binary version %[1]s (15)
97-
│ ├── run "mixed-version 2" (16)
98-
│ ├── restart node 2 with binary version %[1]s (17)
99-
│ ├── restart node 3 with binary version %[1]s (18)
100-
│ ├── restart node 1 with binary version %[1]s (19)
101-
│ └── run "mixed-version 1" (20)
95+
│ ├── restart node 4 with binary version %[1]s (14)
96+
│ ├── run "mixed-version 2" (15)
97+
│ ├── restart node 2 with binary version %[1]s (16)
98+
│ ├── restart node 3 with binary version %[1]s (17)
99+
│ ├── restart node 1 with binary version %[1]s (18)
100+
│ └── run "mixed-version 1" (19)
102101
├── upgrade nodes :1-4 from "%[1]s" to "<current>"
103-
│ ├── restart node 4 with binary version <current> (21)
104-
│ ├── run "mixed-version 1" (22)
105-
│ ├── restart node 1 with binary version <current> (23)
106-
│ ├── restart node 2 with binary version <current> (24)
107-
│ ├── run "mixed-version 2" (25)
108-
│ └── restart node 3 with binary version <current> (26)
109-
├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (27)
110-
└── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (28)
102+
│ ├── restart node 4 with binary version <current> (20)
103+
│ ├── run "mixed-version 1" (21)
104+
│ ├── restart node 1 with binary version <current> (22)
105+
│ ├── restart node 2 with binary version <current> (23)
106+
│ ├── run "mixed-version 2" (24)
107+
│ └── restart node 3 with binary version <current> (25)
108+
├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (26)
109+
└── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (27)
111110
`, predecessorVersion)
112111

113112
expectedPrettyPlan = expectedPrettyPlan[1:] // remove leading newline
@@ -121,7 +120,7 @@ mixed-version test plan for upgrading from %[1]s to <current>:
121120
mvt.OnStartup("startup 2", dummyHook)
122121
plan, err = mvt.plan()
123122
require.NoError(t, err)
124-
requireConcurrentHooks(t, plan.steps[4], "startup 1", "startup 2")
123+
requireConcurrentHooks(t, plan.steps[3], "startup 1", "startup 2")
125124

126125
// Assert that AfterUpgradeFinalized hooks are scheduled to run in
127126
// the last step of the test.
@@ -131,8 +130,8 @@ mixed-version test plan for upgrading from %[1]s to <current>:
131130
mvt.AfterUpgradeFinalized("finalizer 3", dummyHook)
132131
plan, err = mvt.plan()
133132
require.NoError(t, err)
134-
require.Len(t, plan.steps, 10)
135-
requireConcurrentHooks(t, plan.steps[9], "finalizer 1", "finalizer 2", "finalizer 3")
133+
require.Len(t, plan.steps, 9)
134+
requireConcurrentHooks(t, plan.steps[8], "finalizer 1", "finalizer 2", "finalizer 3")
136135
}
137136

138137
// TestDeterministicTestPlan tests that generating a test plan with
@@ -199,15 +198,15 @@ func TestDeterministicHookSeeds(t *testing.T) {
199198

200199
// We can hardcode these paths since we are using a fixed seed in
201200
// these tests.
202-
firstRun := plan.steps[4].(sequentialRunStep).steps[4].(runHookStep)
201+
firstRun := plan.steps[3].(sequentialRunStep).steps[4].(runHookStep)
203202
require.Equal(t, "do something", firstRun.hook.name)
204203
require.NoError(t, firstRun.Run(ctx, nilLogger, nilCluster, emptyHelper))
205204

206-
secondRun := plan.steps[5].(sequentialRunStep).steps[1].(runHookStep)
205+
secondRun := plan.steps[4].(sequentialRunStep).steps[1].(runHookStep)
207206
require.Equal(t, "do something", secondRun.hook.name)
208207
require.NoError(t, secondRun.Run(ctx, nilLogger, nilCluster, emptyHelper))
209208

210-
thirdRun := plan.steps[6].(sequentialRunStep).steps[3].(runHookStep)
209+
thirdRun := plan.steps[5].(sequentialRunStep).steps[3].(runHookStep)
211210
require.Equal(t, "do something", thirdRun.hook.name)
212211
require.NoError(t, thirdRun.Run(ctx, nilLogger, nilCluster, emptyHelper))
213212

pkg/cmd/roachtest/test/test_interface.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,11 @@ import (
1515
"github.com/cockroachdb/cockroach/pkg/util/version"
1616
)
1717

18+
// DefaultCockroachPath is the path where the binary passed to the
19+
// `--cockroach` flag will be made available in every node in the
20+
// cluster.
21+
const DefaultCockroachPath = "./cockroach-default"
22+
1823
// Test is the interface through which roachtests interact with the
1924
// test harness.
2025
type Test interface {

pkg/cmd/roachtest/tests/mixed_version_backup.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -813,7 +813,7 @@ func (sc *systemTableContents) loadShowResults(
813813
}
814814

815815
query := fmt.Sprintf("SELECT * FROM [%s]%s", showStmt, aostFor(timestamp))
816-
showCmd := roachtestutil.NewCommand("%s sql", mixedversion.CurrentCockroachPath).
816+
showCmd := roachtestutil.NewCommand("%s sql", test.DefaultCockroachPath).
817817
Flag("certs-dir", "certs").
818818
Flag("e", fmt.Sprintf("%q", query)).
819819
String()

pkg/sql/tests/rsg_test.go

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -200,6 +200,16 @@ func (db *verifyFormatDB) execWithResettableTimeout(
200200
return &nonCrasher{sql: sql, err: err}
201201
}
202202
return nil
203+
case <-ctx.Done():
204+
// Sanity: The context is cancelled when the test is about to
205+
// timeout. We will log whatever statement we're waiting on for
206+
// debugging purposes. Sometimes queries won't respect
207+
// cancellation due to lib/pq limitations.
208+
t.Logf("Context cancelled while executing: %q", sql)
209+
// We will intentionally retry, which will us to wait for the
210+
// go routine to complete above to avoid leaking it.
211+
retry = true
212+
return nil
203213
case <-time.After(targetDuration):
204214
db.mu.Lock()
205215
defer db.mu.Unlock()
@@ -357,7 +367,9 @@ func TestRandomSyntaxFunctions(t *testing.T) {
357367
case "crdb_internal.reset_sql_stats",
358368
"crdb_internal.check_consistency",
359369
"crdb_internal.request_statement_bundle",
360-
"crdb_internal.reset_activity_tables":
370+
"crdb_internal.reset_activity_tables",
371+
"crdb_internal.revalidate_unique_constraints_in_all_tables",
372+
"crdb_internal.validate_ttl_scheduled_jobs":
361373
// Skipped due to long execution time.
362374
continue
363375
}

0 commit comments

Comments
 (0)