diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index fac1d07a97ad..91dfd4f6f205 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -14,6 +14,7 @@ import ( "fmt" "math" "strings" + "sync/atomic" "testing" "time" @@ -697,7 +698,8 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { // The test uses follower_read_timestamp(). defer utilccl.TestingEnableEnterprise()() - historicalQuery := `SELECT * FROM test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2` + var historicalQuery atomic.Value + historicalQuery.Store(`SELECT * FROM test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2`) recCh := make(chan tracingpb.Recording, 1) tc := testcluster.StartTestCluster(t, 4, @@ -732,7 +734,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { }, SQLExecutor: &sql.ExecutorTestingKnobs{ WithStatementTrace: func(trace tracingpb.Recording, stmt string) { - if stmt == historicalQuery { + if stmt == historicalQuery.Load().(string) { recCh <- trace } }, @@ -786,7 +788,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { // not be executed as a follower read since it attempts to use n2 which // doesn't have a replica any more and then it tries n1 which returns an // updated descriptor. - n4.Exec(t, historicalQuery) + n4.Exec(t, historicalQuery.Load().(string)) // As a sanity check, verify that this was not a follower read. rec := <-recCh require.False(t, kv.OnlyFollowerReads(rec), "query was served through follower reads: %s", rec) @@ -812,7 +814,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { // Run a historical query and assert that it's served from the follower (n3). // n4 should attempt to route to n3 because we pretend n3 has a lower latency // (see testing knob). - n4.Exec(t, historicalQuery) + n4.Exec(t, historicalQuery.Load().(string)) rec = <-recCh // Look at the trace and check that we've served a follower read. @@ -855,8 +857,8 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { // the ReplicaInfo twice for the same range. This allows us to verify that // the cached - in the spanResolverIterator - information is correctly // preserved. - historicalQuery = `SELECT * FROM [SELECT * FROM test WHERE k=2 UNION ALL SELECT * FROM test WHERE k=3] AS OF SYSTEM TIME follower_read_timestamp()` - n4.Exec(t, historicalQuery) + 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()`) + n4.Exec(t, historicalQuery.Load().(string)) rec = <-recCh // Sanity check that the plan was distributed. diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index a72393ac4cd2..86a75fce72b9 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -115,7 +115,6 @@ const ( defaultEncryptionProbability = 1 defaultFIPSProbability = 0 defaultARM64Probability = 0 - defaultCockroachPath = "./cockroach-default" ) type errBinaryOrLibraryNotFound struct { @@ -1347,7 +1346,7 @@ func (c *clusterImpl) FetchLogs(ctx context.Context, l *logger.Logger) error { } } - 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 { + 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 { l.Printf("failed to redact logs: %v", err) if ctx.Err() != nil { return err @@ -1428,7 +1427,7 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger) sec = fmt.Sprintf("--certs-dir=%s", certs) } if err := c.RunE( - ctx, c.Node(node), fmt.Sprintf("%s debug tsdump %s --format=raw > tsdump.gob", defaultCockroachPath, sec), + ctx, c.Node(node), fmt.Sprintf("%s debug tsdump %s --format=raw > tsdump.gob", test.DefaultCockroachPath, sec), ); err != nil { return err } @@ -1499,7 +1498,7 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger, dest // Ignore the files in the the log directory; we pull the logs separately anyway // so this would only cause duplication. excludeFiles := "*.log,*.txt,*.pprof" - cmd := roachtestutil.NewCommand("%s debug zip", defaultCockroachPath). + cmd := roachtestutil.NewCommand("%s debug zip", test.DefaultCockroachPath). Option("include-range-info"). Flag("exclude-files", fmt.Sprintf("'%s'", excludeFiles)). Flag("url", fmt.Sprintf("{pgurl:%d}", i)). @@ -1507,7 +1506,7 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger, dest Arg(zipName). String() if err := c.RunE(ctx, c.Node(i), cmd); err != nil { - l.Printf("%s debug zip failed on node %d: %v", defaultCockroachPath, i, err) + l.Printf("%s debug zip failed on node %d: %v", test.DefaultCockroachPath, i, err) if i < c.spec.NodeCount { continue } @@ -1928,7 +1927,7 @@ func (c *clusterImpl) PutE( } // PutDefaultCockroach uploads the cockroach binary passed in the -// command line to `defaultCockroachPath` in every node in the +// command line to `test.DefaultCockroachPath` in every node in the // cluster. This binary is used by the test runner to collect failure // artifacts since tests are free to upload the cockroach binary they // use to any location they desire. @@ -1936,7 +1935,7 @@ func (c *clusterImpl) PutDefaultCockroach( ctx context.Context, l *logger.Logger, cockroachPath string, ) error { c.status("uploading default cockroach binary to nodes") - return c.PutE(ctx, l, cockroachPath, defaultCockroachPath, c.All()) + return c.PutE(ctx, l, cockroachPath, test.DefaultCockroachPath, c.All()) } // PutLibraries inserts the specified libraries, by name, into all nodes on the cluster diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index 896266cdce78..7caa754eed21 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -111,13 +111,6 @@ const ( // cluster that can use the test fixtures in // `pkg/cmd/roachtest/fixtures`. numNodesInFixtures = 4 - - // CurrentCockroachPath is the path to the binary where the current - // version of cockroach being tested is located. This file is - // uploaded before any user functions are run. The primary use case - // are tests that need long runnig background functions on startup - // (such as running a workload). - CurrentCockroachPath = "./cockroach-current" ) var ( @@ -581,35 +574,6 @@ func (s startStep) Run(ctx context.Context, l *logger.Logger, c cluster.Cluster, return clusterupgrade.StartWithSettings(ctx, l, c, s.crdbNodes, startOpts, clusterSettings...) } -// uploadCurrentVersionStep uploads the current cockroach binary to -// all DB nodes in the test. This is so that startup steps can use -// them (if, for instance, they need to run a workload). The binary -// will be located in `dest`. -type uploadCurrentVersionStep struct { - id int - rt test.Test - crdbNodes option.NodeListOption - dest string -} - -func (s uploadCurrentVersionStep) ID() int { return s.id } -func (s uploadCurrentVersionStep) Background() shouldStop { return nil } - -func (s uploadCurrentVersionStep) Description() string { - return fmt.Sprintf("upload current binary to all cockroach nodes (%v)", s.crdbNodes) -} - -func (s uploadCurrentVersionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, h *Helper, -) error { - _, err := clusterupgrade.UploadVersion(ctx, s.rt, l, c, s.crdbNodes, clusterupgrade.MainVersion) - if err != nil { - return err - } - - return c.RunE(ctx, s.crdbNodes, fmt.Sprintf("mv ./cockroach %s", s.dest)) -} - // waitForStableClusterVersionStep implements the process of waiting // for the `version` cluster setting being the same on all nodes of // the cluster and equal to the binary version of the first node in diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go index 4a2fc7192c7b..7ad48a9d38dc 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go @@ -132,7 +132,6 @@ func (p *testPlanner) initSteps() []testStep { return append( append(steps, - uploadCurrentVersionStep{id: p.nextID(), rt: p.rt, crdbNodes: p.crdbNodes, dest: CurrentCockroachPath}, waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes, timeout: p.options.upgradeTimeout}, preserveDowngradeOptionStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes}, ), diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go index 7b9ee036df5c..d17dc9765d62 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go @@ -69,7 +69,7 @@ func TestTestPlanner(t *testing.T) { plan, err := mvt.plan() require.NoError(t, err) - require.Len(t, plan.steps, 11) + require.Len(t, plan.steps, 10) // Assert on the pretty-printed version of the test plan as that // asserts the ordering of the steps we want to take, and as a bonus @@ -77,37 +77,36 @@ func TestTestPlanner(t *testing.T) { expectedPrettyPlan := fmt.Sprintf(` mixed-version test plan for upgrading from %[1]s to : ├── starting cluster at version "%[1]s" (1) -├── upload current binary to all cockroach nodes (:1-4) (2) -├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (3) -├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (4) -├── run "initialize bank workload" (5) +├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (2) +├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (3) +├── run "initialize bank workload" (4) ├── start background hooks concurrently -│ ├── run "bank workload", after 50ms delay (6) -│ ├── run "rand workload", after 200ms delay (7) -│ └── run "csv server", after 500ms delay (8) +│ ├── run "bank workload", after 50ms delay (5) +│ ├── run "rand workload", after 200ms delay (6) +│ └── run "csv server", after 500ms delay (7) ├── upgrade nodes :1-4 from "%[1]s" to "" -│ ├── restart node 1 with binary version (9) -│ ├── run "mixed-version 1" (10) -│ ├── restart node 4 with binary version (11) -│ ├── restart node 3 with binary version (12) -│ ├── run "mixed-version 2" (13) -│ └── restart node 2 with binary version (14) +│ ├── restart node 1 with binary version (8) +│ ├── run "mixed-version 1" (9) +│ ├── restart node 4 with binary version (10) +│ ├── restart node 3 with binary version (11) +│ ├── run "mixed-version 2" (12) +│ └── restart node 2 with binary version (13) ├── downgrade nodes :1-4 from "" to "%[1]s" -│ ├── restart node 4 with binary version %[1]s (15) -│ ├── run "mixed-version 2" (16) -│ ├── restart node 2 with binary version %[1]s (17) -│ ├── restart node 3 with binary version %[1]s (18) -│ ├── restart node 1 with binary version %[1]s (19) -│ └── run "mixed-version 1" (20) +│ ├── restart node 4 with binary version %[1]s (14) +│ ├── run "mixed-version 2" (15) +│ ├── restart node 2 with binary version %[1]s (16) +│ ├── restart node 3 with binary version %[1]s (17) +│ ├── restart node 1 with binary version %[1]s (18) +│ └── run "mixed-version 1" (19) ├── upgrade nodes :1-4 from "%[1]s" to "" -│ ├── restart node 4 with binary version (21) -│ ├── run "mixed-version 1" (22) -│ ├── restart node 1 with binary version (23) -│ ├── restart node 2 with binary version (24) -│ ├── run "mixed-version 2" (25) -│ └── restart node 3 with binary version (26) -├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (27) -└── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (28) +│ ├── restart node 4 with binary version (20) +│ ├── run "mixed-version 1" (21) +│ ├── restart node 1 with binary version (22) +│ ├── restart node 2 with binary version (23) +│ ├── run "mixed-version 2" (24) +│ └── restart node 3 with binary version (25) +├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (26) +└── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (27) `, predecessorVersion) expectedPrettyPlan = expectedPrettyPlan[1:] // remove leading newline @@ -121,7 +120,7 @@ mixed-version test plan for upgrading from %[1]s to : mvt.OnStartup("startup 2", dummyHook) plan, err = mvt.plan() require.NoError(t, err) - requireConcurrentHooks(t, plan.steps[4], "startup 1", "startup 2") + requireConcurrentHooks(t, plan.steps[3], "startup 1", "startup 2") // Assert that AfterUpgradeFinalized hooks are scheduled to run in // the last step of the test. @@ -131,8 +130,8 @@ mixed-version test plan for upgrading from %[1]s to : mvt.AfterUpgradeFinalized("finalizer 3", dummyHook) plan, err = mvt.plan() require.NoError(t, err) - require.Len(t, plan.steps, 10) - requireConcurrentHooks(t, plan.steps[9], "finalizer 1", "finalizer 2", "finalizer 3") + require.Len(t, plan.steps, 9) + requireConcurrentHooks(t, plan.steps[8], "finalizer 1", "finalizer 2", "finalizer 3") } // TestDeterministicTestPlan tests that generating a test plan with @@ -199,15 +198,15 @@ func TestDeterministicHookSeeds(t *testing.T) { // We can hardcode these paths since we are using a fixed seed in // these tests. - firstRun := plan.steps[4].(sequentialRunStep).steps[4].(runHookStep) + firstRun := plan.steps[3].(sequentialRunStep).steps[4].(runHookStep) require.Equal(t, "do something", firstRun.hook.name) require.NoError(t, firstRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) - secondRun := plan.steps[5].(sequentialRunStep).steps[1].(runHookStep) + secondRun := plan.steps[4].(sequentialRunStep).steps[1].(runHookStep) require.Equal(t, "do something", secondRun.hook.name) require.NoError(t, secondRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) - thirdRun := plan.steps[6].(sequentialRunStep).steps[3].(runHookStep) + thirdRun := plan.steps[5].(sequentialRunStep).steps[3].(runHookStep) require.Equal(t, "do something", thirdRun.hook.name) require.NoError(t, thirdRun.Run(ctx, nilLogger, nilCluster, emptyHelper)) diff --git a/pkg/cmd/roachtest/test/test_interface.go b/pkg/cmd/roachtest/test/test_interface.go index 1047231fba1e..ae20380673f4 100644 --- a/pkg/cmd/roachtest/test/test_interface.go +++ b/pkg/cmd/roachtest/test/test_interface.go @@ -15,6 +15,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/version" ) +// DefaultCockroachPath is the path where the binary passed to the +// `--cockroach` flag will be made available in every node in the +// cluster. +const DefaultCockroachPath = "./cockroach-default" + // Test is the interface through which roachtests interact with the // test harness. type Test interface { diff --git a/pkg/cmd/roachtest/tests/mixed_version_backup.go b/pkg/cmd/roachtest/tests/mixed_version_backup.go index c0c87c9ed59e..675d33ba1aa5 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_backup.go +++ b/pkg/cmd/roachtest/tests/mixed_version_backup.go @@ -813,7 +813,7 @@ func (sc *systemTableContents) loadShowResults( } query := fmt.Sprintf("SELECT * FROM [%s]%s", showStmt, aostFor(timestamp)) - showCmd := roachtestutil.NewCommand("%s sql", mixedversion.CurrentCockroachPath). + showCmd := roachtestutil.NewCommand("%s sql", test.DefaultCockroachPath). Flag("certs-dir", "certs"). Flag("e", fmt.Sprintf("%q", query)). String() diff --git a/pkg/sql/tests/rsg_test.go b/pkg/sql/tests/rsg_test.go index c28470c877ca..969be7d90414 100644 --- a/pkg/sql/tests/rsg_test.go +++ b/pkg/sql/tests/rsg_test.go @@ -200,6 +200,16 @@ func (db *verifyFormatDB) execWithResettableTimeout( return &nonCrasher{sql: sql, err: err} } return nil + case <-ctx.Done(): + // Sanity: The context is cancelled when the test is about to + // timeout. We will log whatever statement we're waiting on for + // debugging purposes. Sometimes queries won't respect + // cancellation due to lib/pq limitations. + t.Logf("Context cancelled while executing: %q", sql) + // We will intentionally retry, which will us to wait for the + // go routine to complete above to avoid leaking it. + retry = true + return nil case <-time.After(targetDuration): db.mu.Lock() defer db.mu.Unlock() @@ -357,7 +367,9 @@ func TestRandomSyntaxFunctions(t *testing.T) { case "crdb_internal.reset_sql_stats", "crdb_internal.check_consistency", "crdb_internal.request_statement_bundle", - "crdb_internal.reset_activity_tables": + "crdb_internal.reset_activity_tables", + "crdb_internal.revalidate_unique_constraints_in_all_tables", + "crdb_internal.validate_ttl_scheduled_jobs": // Skipped due to long execution time. continue }