From 009c53d8e652bcaceff58ed27d666b8df3d0f45c Mon Sep 17 00:00:00 2001 From: DarrylWong Date: Thu, 9 Mar 2023 17:30:29 -0500 Subject: [PATCH 1/3] roachtest: allow tests to specify a cockroach binary to use Currently, roachtests must manually upload their own cockroach binary if needed through the Put API. However, almost all roachtests upload the standard t.Cockroach() binary to ./cockroach on all nodes, resulting in the same Put code being duplicated at the start of most tests. Additionally, to collect artifacts we still need a cockroach binary at a discoverable path, leading to the same binary being copied twice in many cases, see: #97814 This change adds a TestSpec option which lets tests specify a cockroach binary to use. If one is not specified, we now upload the t.Cockroach() binary to ./cockroach. This lets us remove cockroach-default logic for artifacts, and removes the need to manually upload binaries at the start of each test. Release note: None Fixes: #104729 --- pkg/cmd/roachtest/cluster.go | 25 +++++++++++-------- pkg/cmd/roachtest/registry/test_spec.go | 17 +++++++++++++ pkg/cmd/roachtest/test/test_interface.go | 2 +- pkg/cmd/roachtest/test_runner.go | 2 +- pkg/cmd/roachtest/tests/activerecord.go | 1 - .../tests/admission_control_database_drop.go | 1 - .../tests/admission_control_elastic_io.go | 1 - .../admission_control_follower_overload.go | 1 - .../tests/admission_control_index_backfill.go | 1 - .../tests/admission_control_index_overload.go | 1 - .../admission_control_intent_resolution.go | 1 - .../admission_control_multi_store_overload.go | 1 - .../admission_control_multitenant_fairness.go | 1 - .../admission_control_snapshot_overload.go | 1 - .../tests/admission_control_tpcc_overload.go | 1 - pkg/cmd/roachtest/tests/allocation_bench.go | 1 - pkg/cmd/roachtest/tests/allocator.go | 3 --- pkg/cmd/roachtest/tests/alterpk.go | 1 - pkg/cmd/roachtest/tests/asyncpg.go | 1 - pkg/cmd/roachtest/tests/awsdms.go | 1 - pkg/cmd/roachtest/tests/backup.go | 3 --- pkg/cmd/roachtest/tests/build_info.go | 3 --- pkg/cmd/roachtest/tests/cancel.go | 1 - pkg/cmd/roachtest/tests/cdc.go | 7 ------ pkg/cmd/roachtest/tests/cdc_bench.go | 2 -- pkg/cmd/roachtest/tests/clearrange.go | 2 -- pkg/cmd/roachtest/tests/cli.go | 1 - pkg/cmd/roachtest/tests/clock_jump_crash.go | 4 --- pkg/cmd/roachtest/tests/clock_monotonic.go | 4 --- pkg/cmd/roachtest/tests/cluster_init.go | 2 -- pkg/cmd/roachtest/tests/cluster_to_cluster.go | 1 - pkg/cmd/roachtest/tests/connection_latency.go | 5 +--- pkg/cmd/roachtest/tests/copy.go | 1 - pkg/cmd/roachtest/tests/copyfrom.go | 1 - pkg/cmd/roachtest/tests/decommission.go | 20 ++++----------- pkg/cmd/roachtest/tests/decommissionbench.go | 1 - pkg/cmd/roachtest/tests/disagg_rebalance.go | 1 - pkg/cmd/roachtest/tests/disk_full.go | 1 - pkg/cmd/roachtest/tests/disk_stall.go | 1 - pkg/cmd/roachtest/tests/django.go | 1 - pkg/cmd/roachtest/tests/drain.go | 15 ++--------- pkg/cmd/roachtest/tests/drop.go | 1 - pkg/cmd/roachtest/tests/encryption.go | 1 - pkg/cmd/roachtest/tests/event_log.go | 1 - pkg/cmd/roachtest/tests/failover.go | 7 ------ pkg/cmd/roachtest/tests/flowable.go | 1 - pkg/cmd/roachtest/tests/follower_reads.go | 1 - pkg/cmd/roachtest/tests/gopg.go | 1 - pkg/cmd/roachtest/tests/gorm.go | 1 - pkg/cmd/roachtest/tests/gossip.go | 6 ----- pkg/cmd/roachtest/tests/hibernate.go | 1 - pkg/cmd/roachtest/tests/hotspotsplits.go | 1 - pkg/cmd/roachtest/tests/import.go | 5 ---- .../roachtest/tests/import_cancellation.go | 1 - pkg/cmd/roachtest/tests/inconsistency.go | 1 - pkg/cmd/roachtest/tests/indexes.go | 1 - pkg/cmd/roachtest/tests/inverted_index.go | 1 - pkg/cmd/roachtest/tests/jasyncsql.go | 1 - pkg/cmd/roachtest/tests/jepsen.go | 3 --- pkg/cmd/roachtest/tests/knex.go | 1 - pkg/cmd/roachtest/tests/kv.go | 8 ------ pkg/cmd/roachtest/tests/kvbench.go | 3 --- pkg/cmd/roachtest/tests/lease_preferences.go | 1 - pkg/cmd/roachtest/tests/ledger.go | 1 - pkg/cmd/roachtest/tests/libpq.go | 1 - pkg/cmd/roachtest/tests/liquibase.go | 1 - .../tests/loss_of_quorum_recovery.go | 2 -- pkg/cmd/roachtest/tests/many_splits.go | 1 - pkg/cmd/roachtest/tests/mixed_version_cdc.go | 1 - pkg/cmd/roachtest/tests/multitenant.go | 2 -- .../roachtest/tests/multitenant_distsql.go | 1 - .../tests/multitenant_shared_process.go | 1 - pkg/cmd/roachtest/tests/multitenant_tpch.go | 1 - pkg/cmd/roachtest/tests/mvcc_gc.go | 1 - pkg/cmd/roachtest/tests/network.go | 2 -- pkg/cmd/roachtest/tests/network_logging.go | 1 - pkg/cmd/roachtest/tests/nodejs_postgres.go | 4 +-- pkg/cmd/roachtest/tests/npgsql.go | 1 - pkg/cmd/roachtest/tests/pgjdbc.go | 1 - pkg/cmd/roachtest/tests/pgx.go | 1 - pkg/cmd/roachtest/tests/pop.go | 1 - pkg/cmd/roachtest/tests/process_lock.go | 1 - pkg/cmd/roachtest/tests/psycopg.go | 1 - .../roachtest/tests/query_comparison_util.go | 2 -- pkg/cmd/roachtest/tests/queue.go | 1 - pkg/cmd/roachtest/tests/quit.go | 1 - pkg/cmd/roachtest/tests/rapid_restart.go | 1 - pkg/cmd/roachtest/tests/rebalance_load.go | 1 - pkg/cmd/roachtest/tests/replicagc.go | 1 - pkg/cmd/roachtest/tests/restart.go | 1 - pkg/cmd/roachtest/tests/restore.go | 1 - pkg/cmd/roachtest/tests/roachmart.go | 1 - pkg/cmd/roachtest/tests/roachtest.go | 1 - pkg/cmd/roachtest/tests/ruby_pg.go | 1 - pkg/cmd/roachtest/tests/rust_postgres.go | 1 - pkg/cmd/roachtest/tests/schemachange.go | 2 -- .../tests/schemachange_random_load.go | 1 - pkg/cmd/roachtest/tests/sequelize.go | 1 - pkg/cmd/roachtest/tests/slow_drain.go | 7 ++---- pkg/cmd/roachtest/tests/smoketest_secure.go | 2 -- pkg/cmd/roachtest/tests/split.go | 7 +++--- pkg/cmd/roachtest/tests/sqlalchemy.go | 1 - pkg/cmd/roachtest/tests/sqlsmith.go | 1 - pkg/cmd/roachtest/tests/sstable_corruption.go | 1 - pkg/cmd/roachtest/tests/status_server.go | 1 - pkg/cmd/roachtest/tests/sysbench.go | 1 - pkg/cmd/roachtest/tests/tlp.go | 2 -- pkg/cmd/roachtest/tests/tombstones.go | 1 - pkg/cmd/roachtest/tests/tpcc.go | 8 ------ pkg/cmd/roachtest/tests/tpcdsvec.go | 1 - pkg/cmd/roachtest/tests/tpce.go | 7 +++--- pkg/cmd/roachtest/tests/tpch_concurrency.go | 8 +++--- pkg/cmd/roachtest/tests/tpchbench.go | 1 - pkg/cmd/roachtest/tests/tpchvec.go | 1 - pkg/cmd/roachtest/tests/typeorm.go | 1 - pkg/cmd/roachtest/tests/ycsb.go | 1 - 116 files changed, 55 insertions(+), 229 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 6e45d752e471..19d3d922ccfc 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1749,16 +1749,21 @@ func (c *clusterImpl) PutE( return errors.Wrap(roachprod.Put(ctx, l, c.MakeNodes(nodes...), src, dest, true /* useTreeDist */), "cluster.PutE") } -// PutDefaultCockroach uploads the cockroach binary passed 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. -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, test.DefaultCockroachPath, c.All()) +// PutCockroach checks if a test specifies a cockroach binary to upload to all +// nodes in the cluster. By default, we randomly upload a binary with or without +// runtime assertions enabled. Note that we upload to all nodes even if they +// don't use the binary, so that the test runner can always fetch logs. +func (c *clusterImpl) PutCockroach(ctx context.Context, l *logger.Logger, t *testImpl) error { + switch t.spec.CockroachBinary { + case registry.RandomizedCockroach: + return c.PutE(ctx, l, t.Cockroach(), test.DefaultCockroachPath, c.All()) + case registry.StandardCockroach: + return c.PutE(ctx, l, t.StandardCockroach(), test.DefaultCockroachPath, c.All()) + case registry.RuntimeAssertionsCockroach: + return c.PutE(ctx, l, t.RuntimeAssertionsCockroach(), test.DefaultCockroachPath, c.All()) + default: + return errors.Errorf("Specified cockroach binary does not exist.") + } } // PutLibraries inserts the specified libraries, by name, into all nodes on the cluster diff --git a/pkg/cmd/roachtest/registry/test_spec.go b/pkg/cmd/roachtest/registry/test_spec.go index bb2cdd6904af..52bd8934655d 100644 --- a/pkg/cmd/roachtest/registry/test_spec.go +++ b/pkg/cmd/roachtest/registry/test_spec.go @@ -132,6 +132,13 @@ type TestSpec struct { // ExtraLabels are test-specific labels that will be added to the Github // issue created when a failure occurs, in addition to default labels. ExtraLabels []string + + // CockroachBinary is the cockroach binary that will be uploaded + // to every node in the cluster at the start of the test. We upload to + // every node so that we can fetch logs in the case of a failure. + // If one is not specified, the default behavior is to upload + // a binary with the crdb_test flag randomly enabled or disabled. + CockroachBinary ClusterCockroachBinary } // PostValidation is a type of post-validation that runs after a test completes. @@ -426,3 +433,13 @@ func setToString(validValues []string, m map[string]struct{}) string { } return strings.Join(elems, ",") } + +// ClusterCockroachBinary specifies the type of cockroach binaries that +// can be uploaded to the cluster. +type ClusterCockroachBinary int + +const ( + RandomizedCockroach ClusterCockroachBinary = iota + StandardCockroach + RuntimeAssertionsCockroach +) diff --git a/pkg/cmd/roachtest/test/test_interface.go b/pkg/cmd/roachtest/test/test_interface.go index 19d09f6ef7c2..cdf93721f1cd 100644 --- a/pkg/cmd/roachtest/test/test_interface.go +++ b/pkg/cmd/roachtest/test/test_interface.go @@ -18,7 +18,7 @@ import ( // 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" +const DefaultCockroachPath = "./cockroach" // EnvAssertionsEnabledSeed is the name of the environment variable // that, when set, causes roachtest to use a binary with runtime diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 8f1d9b8014f3..efa3aa94ea46 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -746,7 +746,7 @@ func (r *testRunner) runWorker( var setupErr error if c.spec.NodeCount > 0 { // skip during tests - setupErr = c.PutDefaultCockroach(ctx, l, t.Cockroach()) + setupErr = c.PutCockroach(ctx, l, t) } if setupErr == nil { setupErr = c.PutLibraries(ctx, "./lib", t.spec.NativeLibs) diff --git a/pkg/cmd/roachtest/tests/activerecord.go b/pkg/cmd/roachtest/tests/activerecord.go index 7e86888a68a9..fcbcfb31ecf8 100644 --- a/pkg/cmd/roachtest/tests/activerecord.go +++ b/pkg/cmd/roachtest/tests/activerecord.go @@ -48,7 +48,6 @@ func registerActiveRecord(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/admission_control_database_drop.go b/pkg/cmd/roachtest/tests/admission_control_database_drop.go index 5dbaf1b2491b..8432910b8490 100644 --- a/pkg/cmd/roachtest/tests/admission_control_database_drop.go +++ b/pkg/cmd/roachtest/tests/admission_control_database_drop.go @@ -197,7 +197,6 @@ func registerDatabaseDrop(r registry.Registry) { // test and use disk snapshots? runTPCE(ctx, t, c, tpceOptions{ start: func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) startOpts := option.DefaultStartOptsNoBackups() settings := install.MakeClusterSettings(install.NumRacksOption(crdbNodes)) if err := c.StartE(ctx, t.L(), startOpts, settings, c.Range(1, crdbNodes)); err != nil { diff --git a/pkg/cmd/roachtest/tests/admission_control_elastic_io.go b/pkg/cmd/roachtest/tests/admission_control_elastic_io.go index 2fe2e473b14a..69bbe0ba0ff3 100644 --- a/pkg/cmd/roachtest/tests/admission_control_elastic_io.go +++ b/pkg/cmd/roachtest/tests/admission_control_elastic_io.go @@ -68,7 +68,6 @@ func registerElasticIO(r registry.Registry) { require.NoError(t, err) statCollector := clusterstats.NewStatsCollector(ctx, promClient) - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, crdbNodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workAndPromNode)) startOpts := option.DefaultStartOptsNoBackups() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, diff --git a/pkg/cmd/roachtest/tests/admission_control_follower_overload.go b/pkg/cmd/roachtest/tests/admission_control_follower_overload.go index 4b5697d58256..a70e64e6522e 100644 --- a/pkg/cmd/roachtest/tests/admission_control_follower_overload.go +++ b/pkg/cmd/roachtest/tests/admission_control_follower_overload.go @@ -147,7 +147,6 @@ func runAdmissionControlFollowerOverload( phaseDuration := time.Hour nodes := c.Range(1, 3) - c.Put(ctx, t.Cockroach(), "cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), nodes) db := c.Conn(ctx, t.L(), 1) require.NoError(t, WaitFor3XReplication(ctx, t, db)) diff --git a/pkg/cmd/roachtest/tests/admission_control_index_backfill.go b/pkg/cmd/roachtest/tests/admission_control_index_backfill.go index 223f86df7d48..10f486447334 100644 --- a/pkg/cmd/roachtest/tests/admission_control_index_backfill.go +++ b/pkg/cmd/roachtest/tests/admission_control_index_backfill.go @@ -151,7 +151,6 @@ func registerIndexBackfill(r registry.Registry) { // large index backfills while it's running. runTPCE(ctx, t, c, tpceOptions{ start: func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) startOpts := option.DefaultStartOptsNoBackups() settings := install.MakeClusterSettings(install.NumRacksOption(crdbNodes)) if err := c.StartE(ctx, t.L(), startOpts, settings, c.Range(1, crdbNodes)); err != nil { diff --git a/pkg/cmd/roachtest/tests/admission_control_index_overload.go b/pkg/cmd/roachtest/tests/admission_control_index_overload.go index 7f1d23fe153d..dd04d42e2b9b 100644 --- a/pkg/cmd/roachtest/tests/admission_control_index_overload.go +++ b/pkg/cmd/roachtest/tests/admission_control_index_overload.go @@ -48,7 +48,6 @@ func registerIndexOverload(r registry.Registry) { crdbNodes := c.Spec().NodeCount - 1 workloadNode := c.Spec().NodeCount - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), c.Range(1, crdbNodes)) { diff --git a/pkg/cmd/roachtest/tests/admission_control_intent_resolution.go b/pkg/cmd/roachtest/tests/admission_control_intent_resolution.go index 1ceecb49d82e..cba5c4c72286 100644 --- a/pkg/cmd/roachtest/tests/admission_control_intent_resolution.go +++ b/pkg/cmd/roachtest/tests/admission_control_intent_resolution.go @@ -67,7 +67,6 @@ func registerIntentResolutionOverload(r registry.Registry) { require.NoError(t, err) statCollector := clusterstats.NewStatsCollector(ctx, promClient) - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, crdbNodes)) startOpts := option.DefaultStartOptsNoBackups() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--vmodule=io_load_listener=2") diff --git a/pkg/cmd/roachtest/tests/admission_control_multi_store_overload.go b/pkg/cmd/roachtest/tests/admission_control_multi_store_overload.go index 4705412b3586..4d71cd7eaf1a 100644 --- a/pkg/cmd/roachtest/tests/admission_control_multi_store_overload.go +++ b/pkg/cmd/roachtest/tests/admission_control_multi_store_overload.go @@ -26,7 +26,6 @@ import ( func registerMultiStoreOverload(r registry.Registry) { runKV := func(ctx context.Context, t test.Test, c cluster.Cluster) { nodes := c.Spec().NodeCount - 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) startOpts := option.DefaultStartOptsNoBackups() startOpts.RoachprodOpts.StoreCount = 2 diff --git a/pkg/cmd/roachtest/tests/admission_control_multitenant_fairness.go b/pkg/cmd/roachtest/tests/admission_control_multitenant_fairness.go index cbf0a27fecc9..a43f183f1bc1 100644 --- a/pkg/cmd/roachtest/tests/admission_control_multitenant_fairness.go +++ b/pkg/cmd/roachtest/tests/admission_control_multitenant_fairness.go @@ -144,7 +144,6 @@ func runMultiTenantFairness( } t.L().Printf("starting cockroach securely (<%s)", time.Minute) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(install.SecureOption(true)), diff --git a/pkg/cmd/roachtest/tests/admission_control_snapshot_overload.go b/pkg/cmd/roachtest/tests/admission_control_snapshot_overload.go index a4954e0b2ff1..d0ad771b3195 100644 --- a/pkg/cmd/roachtest/tests/admission_control_snapshot_overload.go +++ b/pkg/cmd/roachtest/tests/admission_control_snapshot_overload.go @@ -52,7 +52,6 @@ func registerSnapshotOverload(r registry.Registry) { t.Fatalf("expected at least 4 nodes, found %d", c.Spec().NodeCount) } - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) crdbNodes := c.Spec().NodeCount - 1 workloadNode := crdbNodes + 1 for i := 1; i <= crdbNodes; i++ { diff --git a/pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go b/pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go index 2b4676dd5088..c982a4ddb392 100644 --- a/pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go +++ b/pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go @@ -198,7 +198,6 @@ func registerTPCCSevereOverload(r registry.Registry) { roachNodes := c.Range(1, c.Spec().NodeCount-1) workloadNode := c.Spec().NodeCount - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes) t.Status("initializing (~1h)") diff --git a/pkg/cmd/roachtest/tests/allocation_bench.go b/pkg/cmd/roachtest/tests/allocation_bench.go index adc61c2e43af..0e8a4a6e9284 100644 --- a/pkg/cmd/roachtest/tests/allocation_bench.go +++ b/pkg/cmd/roachtest/tests/allocation_bench.go @@ -274,7 +274,6 @@ func setupAllocationBench( ctx context.Context, t test.Test, c cluster.Cluster, spec allocationBenchSpec, ) (clusterstats.StatCollector, func(context.Context)) { workloadNode := c.Spec().NodeCount - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workloadNode)) t.Status("starting cluster") for i := 1; i <= spec.nodes; i++ { diff --git a/pkg/cmd/roachtest/tests/allocator.go b/pkg/cmd/roachtest/tests/allocator.go index a69c60e953c1..96bdb86f7ec9 100644 --- a/pkg/cmd/roachtest/tests/allocator.go +++ b/pkg/cmd/roachtest/tests/allocator.go @@ -37,8 +37,6 @@ const allocatorStableSeconds = 120 func registerAllocator(r registry.Registry) { runAllocator := func(ctx context.Context, t test.Test, c cluster.Cluster, start int, maxStdDev float64) { - c.Put(ctx, t.Cockroach(), "./cockroach") - // Put away one node to be the stats collector. nodes := c.Spec().NodeCount - 1 @@ -346,7 +344,6 @@ func runWideReplication(ctx context.Context, t test.Test, c cluster.Cluster) { t.Fatalf("9-node cluster required") } - c.Put(ctx, t.Cockroach(), "./cockroach") startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.ExtraArgs = []string{"--vmodule=replicate_queue=6"} settings := install.MakeClusterSettings() diff --git a/pkg/cmd/roachtest/tests/alterpk.go b/pkg/cmd/roachtest/tests/alterpk.go index edc842c144e1..a620bdf2e854 100644 --- a/pkg/cmd/roachtest/tests/alterpk.go +++ b/pkg/cmd/roachtest/tests/alterpk.go @@ -30,7 +30,6 @@ func registerAlterPK(r registry.Registry) { roachNodes := c.Range(1, c.Spec().NodeCount-1) loadNode := c.Node(c.Spec().NodeCount) t.Status("copying binaries") - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", loadNode) t.Status("starting cockroach nodes") diff --git a/pkg/cmd/roachtest/tests/asyncpg.go b/pkg/cmd/roachtest/tests/asyncpg.go index 24a52667686f..d27fa79e9d93 100644 --- a/pkg/cmd/roachtest/tests/asyncpg.go +++ b/pkg/cmd/roachtest/tests/asyncpg.go @@ -46,7 +46,6 @@ func registerAsyncpg(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) // This test assumes that multiple_active_portals_enabled is false, but through // metamorphic constants, it is possible for them to be enabled. We disable diff --git a/pkg/cmd/roachtest/tests/awsdms.go b/pkg/cmd/roachtest/tests/awsdms.go index 68a5b86ade67..47b0d7c1b242 100644 --- a/pkg/cmd/roachtest/tests/awsdms.go +++ b/pkg/cmd/roachtest/tests/awsdms.go @@ -563,7 +563,6 @@ func setupAWSDMS( func setupCockroachDBCluster(ctx context.Context, t test.Test, c cluster.Cluster) func() error { return func() error { t.L().Printf("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) db := c.Conn(ctx, t.L(), 1) diff --git a/pkg/cmd/roachtest/tests/backup.go b/pkg/cmd/roachtest/tests/backup.go index 76da662ff014..448cba34045a 100644 --- a/pkg/cmd/roachtest/tests/backup.go +++ b/pkg/cmd/roachtest/tests/backup.go @@ -96,8 +96,6 @@ func importBankDataSplit( ) string { dest := destinationName(c) - cockroach := t.Cockroach() - c.Put(ctx, cockroach, "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") // NB: starting the cluster creates the logs dir as a side effect, @@ -745,7 +743,6 @@ func runBackupMVCCRangeTombstones( ctx context.Context, t test.Test, c cluster.Cluster, config mvccRangeTombstoneConfig, ) { if !config.skipClusterSetup { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") // required for tpch c.Start(ctx, t.L(), maybeUseMemoryBudget(t, 50), install.MakeClusterSettings()) } diff --git a/pkg/cmd/roachtest/tests/build_info.go b/pkg/cmd/roachtest/tests/build_info.go index 386622158572..8ecfcf0c176d 100644 --- a/pkg/cmd/roachtest/tests/build_info.go +++ b/pkg/cmd/roachtest/tests/build_info.go @@ -25,7 +25,6 @@ import ( // RunBuildInfo is a test that sanity checks the build info. func RunBuildInfo(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) var details serverpb.DetailsResponse @@ -63,8 +62,6 @@ func RunBuildAnalyze(ctx context.Context, t test.Test, c cluster.Cluster) { t.Skip("local execution not supported") } - c.Put(ctx, t.Cockroach(), "./cockroach") - // 1. Check for executable stack. // // Executable stack memory is a security risk (not a vulnerability diff --git a/pkg/cmd/roachtest/tests/cancel.go b/pkg/cmd/roachtest/tests/cancel.go index e72ed262002c..f1b028b5da95 100644 --- a/pkg/cmd/roachtest/tests/cancel.go +++ b/pkg/cmd/roachtest/tests/cancel.go @@ -44,7 +44,6 @@ import ( // replaced with unit tests. func registerCancel(r registry.Registry) { runCancel := func(ctx context.Context, t test.Test, c cluster.Cluster, tpchQueriesToRun []int, useDistsql bool) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) m := c.NewMonitor(ctx, c.All()) diff --git a/pkg/cmd/roachtest/tests/cdc.go b/pkg/cmd/roachtest/tests/cdc.go index a4548de6a797..249693db0707 100644 --- a/pkg/cmd/roachtest/tests/cdc.go +++ b/pkg/cmd/roachtest/tests/cdc.go @@ -619,10 +619,6 @@ func newCDCTester(ctx context.Context, t test.Test, c cluster.Cluster) cdcTester settings.Env = append(settings.Env, envVars...) - // Allow cockroach with runtime assertions enabled unless this is a - // performance test. - cockroach := t.Cockroach() - c.Put(ctx, cockroach, "./cockroach") c.Start(ctx, t.L(), startOpts, settings, tester.crdbNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", tester.workloadNode) tester.startGrafana() @@ -666,7 +662,6 @@ func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) { c.Run(ctx, c.All(), `mkdir -p logs`) crdbNodes, workloadNode, kafkaNode := c.Range(1, c.Spec().NodeCount-1), c.Node(c.Spec().NodeCount), c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", workloadNode) startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, @@ -835,7 +830,6 @@ func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) { // compatibility within a topic). func runCDCSchemaRegistry(ctx context.Context, t test.Test, c cluster.Cluster) { crdbNodes, kafkaNode := c.Node(1), c.Node(1) - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), crdbNodes) kafka := kafkaManager{ t: t, @@ -951,7 +945,6 @@ func runCDCKafkaAuth(ctx context.Context, t test.Test, c cluster.Cluster) { } crdbNodes, kafkaNode := c.Range(1, lastCrdbNode), c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), crdbNodes) kafka := kafkaManager{ diff --git a/pkg/cmd/roachtest/tests/cdc_bench.go b/pkg/cmd/roachtest/tests/cdc_bench.go index eee6809dcbbf..03912746b21f 100644 --- a/pkg/cmd/roachtest/tests/cdc_bench.go +++ b/pkg/cmd/roachtest/tests/cdc_bench.go @@ -260,7 +260,6 @@ func runCDCBenchScan( t.Fatalf("unknown protocol %q", protocol) } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, nData) m := c.NewMonitor(ctx, nData.Merge(nCoord)) @@ -432,7 +431,6 @@ func runCDCBenchWorkload( t.Fatalf("unknown server type %q", server) } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, nData) m := c.NewMonitor(ctx, nData.Merge(nCoord)) diff --git a/pkg/cmd/roachtest/tests/clearrange.go b/pkg/cmd/roachtest/tests/clearrange.go index 2500191806c7..0c3a29423b7b 100644 --- a/pkg/cmd/roachtest/tests/clearrange.go +++ b/pkg/cmd/roachtest/tests/clearrange.go @@ -63,8 +63,6 @@ func registerClearRange(r registry.Registry) { } func runClearRange(ctx context.Context, t test.Test, c cluster.Cluster, aggressiveChecks bool) { - c.Put(ctx, t.Cockroach(), "./cockroach") - t.Status("restoring fixture") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) m := c.NewMonitor(ctx) diff --git a/pkg/cmd/roachtest/tests/cli.go b/pkg/cmd/roachtest/tests/cli.go index 7bd12b7b5ba6..df23f2d6138c 100644 --- a/pkg/cmd/roachtest/tests/cli.go +++ b/pkg/cmd/roachtest/tests/cli.go @@ -25,7 +25,6 @@ import ( ) func runCLINodeStatus(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, 3)) db := c.Conn(ctx, t.L(), 1) diff --git a/pkg/cmd/roachtest/tests/clock_jump_crash.go b/pkg/cmd/roachtest/tests/clock_jump_crash.go index 17e930644ddc..912328aa0a91 100644 --- a/pkg/cmd/roachtest/tests/clock_jump_crash.go +++ b/pkg/cmd/roachtest/tests/clock_jump_crash.go @@ -36,10 +36,6 @@ func runClockJump(ctx context.Context, t test.Test, c cluster.Cluster, tc clockJ t.Fatal(err) } - if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) - } - c.Wipe(ctx, false /* preserveCerts */) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) db := c.Conn(ctx, t.L(), c.Spec().NodeCount) diff --git a/pkg/cmd/roachtest/tests/clock_monotonic.go b/pkg/cmd/roachtest/tests/clock_monotonic.go index b065a008452b..905cb76fc099 100644 --- a/pkg/cmd/roachtest/tests/clock_monotonic.go +++ b/pkg/cmd/roachtest/tests/clock_monotonic.go @@ -38,10 +38,6 @@ func runClockMonotonicity( t.Fatal(err) } - if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) - } - c.Wipe(ctx, false /* preserveCerts */) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) db := c.Conn(ctx, t.L(), c.Spec().NodeCount) diff --git a/pkg/cmd/roachtest/tests/cluster_init.go b/pkg/cmd/roachtest/tests/cluster_init.go index f97bdfe00580..196994fc2567 100644 --- a/pkg/cmd/roachtest/tests/cluster_init.go +++ b/pkg/cmd/roachtest/tests/cluster_init.go @@ -31,8 +31,6 @@ import ( ) func runClusterInit(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") - // We start all nodes with the same join flags and then issue an "init" // command to one of the nodes. We do this twice, since roachtest has some // special casing for the first node in a cluster (the join flags of all nodes diff --git a/pkg/cmd/roachtest/tests/cluster_to_cluster.go b/pkg/cmd/roachtest/tests/cluster_to_cluster.go index 4f835ceab3a4..0148744ca468 100644 --- a/pkg/cmd/roachtest/tests/cluster_to_cluster.go +++ b/pkg/cmd/roachtest/tests/cluster_to_cluster.go @@ -493,7 +493,6 @@ func (rd *replicationDriver) setupC2C( require.NotEqual(t, "", rd.rs.multiregion.workloadNodeZone) } - c.Put(ctx, t.Cockroach(), "./cockroach") srcCluster := c.Range(1, rd.rs.srcNodes) dstCluster := c.Range(rd.rs.srcNodes+1, rd.rs.srcNodes+rd.rs.dstNodes) workloadNode := c.Node(rd.rs.srcNodes + rd.rs.dstNodes + 1) diff --git a/pkg/cmd/roachtest/tests/connection_latency.go b/pkg/cmd/roachtest/tests/connection_latency.go index 07fa0cbd6bca..3af52b96a12c 100644 --- a/pkg/cmd/roachtest/tests/connection_latency.go +++ b/pkg/cmd/roachtest/tests/connection_latency.go @@ -34,10 +34,7 @@ const ( func runConnectionLatencyTest( ctx context.Context, t test.Test, c cluster.Cluster, numNodes int, numZones int, password bool, ) { - err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach") - require.NoError(t, err) - - err = c.PutE(ctx, t.L(), t.DeprecatedWorkload(), "./workload") + err := c.PutE(ctx, t.L(), t.DeprecatedWorkload(), "./workload") require.NoError(t, err) settings := install.MakeClusterSettings(install.SecureOption(true)) diff --git a/pkg/cmd/roachtest/tests/copy.go b/pkg/cmd/roachtest/tests/copy.go index a8152c2c2e5c..c0898365dba0 100644 --- a/pkg/cmd/roachtest/tests/copy.go +++ b/pkg/cmd/roachtest/tests/copy.go @@ -47,7 +47,6 @@ func registerCopy(r registry.Registry) { const rowOverheadEstimate = 160 const rowEstimate = rowOverheadEstimate + payload - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.All()) // We run this without metamorphic constants as kv-batch-size = 1 makes // this test take far too long to complete. diff --git a/pkg/cmd/roachtest/tests/copyfrom.go b/pkg/cmd/roachtest/tests/copyfrom.go index 3cd6943606f6..85ec62fd49e8 100644 --- a/pkg/cmd/roachtest/tests/copyfrom.go +++ b/pkg/cmd/roachtest/tests/copyfrom.go @@ -136,7 +136,6 @@ func runCopyFromPG(ctx context.Context, t test.Test, c cluster.Cluster, sf int) } func runCopyFromCRDB(ctx context.Context, t test.Test, c cluster.Cluster, sf int, atomic bool) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) startOpts := option.DefaultStartOpts() // Enable the verbose logging on relevant files to have better understanding // in case the test fails. diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index 597d26fffba9..4c7fc60cdcab 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -147,7 +147,6 @@ func runDrainAndDecommission( t.Fatal("improper configuration: replication factor greater than number of nodes in the test") } pinnedNode := 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) for i := 1; i <= nodes; i++ { c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Node(i)) } @@ -251,7 +250,6 @@ func runDecommission( // node1 is kept pinned (i.e. not decommissioned/restarted), and is the node // through which we run the workload and other queries. pinnedNode := 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(pinnedNode)) for i := 1; i <= nodes; i++ { @@ -389,7 +387,6 @@ func runDecommission( // those operations. We then fully decommission nodes, verifying it's an // irreversible operation. func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") settings := install.MakeClusterSettings() settings.Env = append(settings.Env, "COCKROACH_SCAN_MAX_IDLE_TIME=5ms") @@ -999,16 +996,12 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) decommNodeID = numNodes decommNode = c.Node(decommNodeID) ) - - err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", c.All()) - require.NoError(t, err) - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) h := newDecommTestHelper(t, c) run := func(db *gosql.DB, query string) error { - _, err = db.ExecContext(ctx, query) + _, err := db.ExecContext(ctx, query) t.L().Printf("run: %s\n", query) return err } @@ -1018,11 +1011,11 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) defer db.Close() // Increase the speed of decommissioning. - err = run(db, `SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`) + err := run(db, `SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`) require.NoError(t, err) // Wait for initial up-replication. - err := WaitFor3XReplication(ctx, t, db) + err = WaitFor3XReplication(ctx, t, db) require.NoError(t, err) } @@ -1093,13 +1086,10 @@ func runDecommissionSlow(ctx context.Context, t test.Test, c cluster.Cluster) { var verboseStoreLogRe = regexp.MustCompile("possible decommission stall detected") - err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", c.All()) - require.NoError(t, err) - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) run := func(db *gosql.DB, query string) { - _, err = db.ExecContext(ctx, query) + _, err := db.ExecContext(ctx, query) require.NoError(t, err) t.L().Printf("run: %s\n", query) } @@ -1144,7 +1134,7 @@ func runDecommissionSlow(ctx context.Context, t test.Test, c cluster.Cluster) { t.Status("checking for decommissioning replicas report...") testutils.SucceedsWithin(t, func() error { for nodeID := 1; nodeID <= numNodes; nodeID++ { - if err = c.RunE(ctx, + if err := c.RunE(ctx, c.Node(nodeID), fmt.Sprintf("grep -q '%s' logs/cockroach.log", verboseStoreLogRe), ); err == nil { diff --git a/pkg/cmd/roachtest/tests/decommissionbench.go b/pkg/cmd/roachtest/tests/decommissionbench.go index abff5495edac..139e1bef4ec3 100644 --- a/pkg/cmd/roachtest/tests/decommissionbench.go +++ b/pkg/cmd/roachtest/tests/decommissionbench.go @@ -373,7 +373,6 @@ func setupDecommissionBench( workloadNode, pinnedNode int, importCmd string, ) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workloadNode)) for i := 1; i <= benchSpec.nodes; i++ { // Don't start a scheduled backup as this roachtest reports to roachperf. diff --git a/pkg/cmd/roachtest/tests/disagg_rebalance.go b/pkg/cmd/roachtest/tests/disagg_rebalance.go index 83c7fb51b3f5..fb8c34c4c9ad 100644 --- a/pkg/cmd/roachtest/tests/disagg_rebalance.go +++ b/pkg/cmd/roachtest/tests/disagg_rebalance.go @@ -41,7 +41,6 @@ func registerDisaggRebalance(r registry.Registry) { if c.Cloud() != spec.AWS { t.Skip("disagg-rebalance is only configured to run on AWS") } - c.Put(ctx, t.Cockroach(), "./cockroach") s3dir := fmt.Sprintf("s3://%s/disagg-rebalance/%s?AUTH=implicit", testutils.BackupTestingBucketLongTTL(), c.Name()) startOpts := option.DefaultStartOptsNoBackups() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, fmt.Sprintf("--experimental-shared-storage=%s", s3dir)) diff --git a/pkg/cmd/roachtest/tests/disk_full.go b/pkg/cmd/roachtest/tests/disk_full.go index 49f3c0e3940f..8eb02c31ae19 100644 --- a/pkg/cmd/roachtest/tests/disk_full.go +++ b/pkg/cmd/roachtest/tests/disk_full.go @@ -40,7 +40,6 @@ func registerDiskFull(r registry.Registry) { } nodes := c.Spec().NodeCount - 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, c.Spec().NodeCount)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, nodes)) // Node 1 will soon be killed, when the ballast file fills up its disk. To diff --git a/pkg/cmd/roachtest/tests/disk_stall.go b/pkg/cmd/roachtest/tests/disk_stall.go index 184f538e8d6e..fbfe94ec4484 100644 --- a/pkg/cmd/roachtest/tests/disk_stall.go +++ b/pkg/cmd/roachtest/tests/disk_stall.go @@ -92,7 +92,6 @@ func runDiskStalledDetection( defer s.Cleanup(ctx) t.Status("starting cluster") - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), startOpts, startSettings, c.Range(1, 3)) // Assert the process monotonic times are as expected. diff --git a/pkg/cmd/roachtest/tests/django.go b/pkg/cmd/roachtest/tests/django.go index 1575d36a6cd9..9929ddc78b03 100644 --- a/pkg/cmd/roachtest/tests/django.go +++ b/pkg/cmd/roachtest/tests/django.go @@ -44,7 +44,6 @@ func registerDjango(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/drain.go b/pkg/cmd/roachtest/tests/drain.go index 13370c1263e3..fb1a94e02e13 100644 --- a/pkg/cmd/roachtest/tests/drain.go +++ b/pkg/cmd/roachtest/tests/drain.go @@ -318,16 +318,11 @@ func runWarningForConnWait(ctx context.Context, t test.Test, c cluster.Cluster) // runClusterNotAtQuorum is to verify that draining works even when the cluster // is not at quorum. func runClusterNotAtQuorum(ctx context.Context, t test.Test, c cluster.Cluster) { - err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", c.All()) - if err != nil { - t.Fatal(err) - } - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) db := c.Conn(ctx, t.L(), 1) defer func() { _ = db.Close() }() - err = WaitFor3XReplication(ctx, t, db) + err := WaitFor3XReplication(ctx, t, db) require.NoError(t, err) stopOpts := option.DefaultStopOpts() @@ -358,12 +353,6 @@ func prepareCluster( connectionWait time.Duration, queryWait time.Duration, ) { - var err error - err = c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", c.All()) - if err != nil { - t.Fatal(err) - } - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) db := c.Conn(ctx, t.L(), 1) @@ -376,7 +365,7 @@ func prepareCluster( fmt.Sprintf("SET CLUSTER SETTING server.shutdown.connections.timeout = '%fs'", connectionWait.Seconds()), } for _, stmt := range waitPhasesSettingStmts { - _, err = db.ExecContext(ctx, stmt) + _, err := db.ExecContext(ctx, stmt) require.NoError(t, err, "cannot set cluster setting") } } diff --git a/pkg/cmd/roachtest/tests/drop.go b/pkg/cmd/roachtest/tests/drop.go index 0cd4ca1f8fe3..df3316b51f17 100644 --- a/pkg/cmd/roachtest/tests/drop.go +++ b/pkg/cmd/roachtest/tests/drop.go @@ -35,7 +35,6 @@ func registerDrop(r registry.Registry) { // rows). Next, it issues a `DROP` for the whole database, and sets the GC TTL // to one second. runDrop := func(ctx context.Context, t test.Test, c cluster.Cluster, warehouses, nodes int) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Range(1, nodes)) settings := install.MakeClusterSettings() settings.Env = append(settings.Env, "COCKROACH_MEMPROF_INTERVAL=15s") diff --git a/pkg/cmd/roachtest/tests/encryption.go b/pkg/cmd/roachtest/tests/encryption.go index 5021974afc96..60831e52a1d0 100644 --- a/pkg/cmd/roachtest/tests/encryption.go +++ b/pkg/cmd/roachtest/tests/encryption.go @@ -28,7 +28,6 @@ func registerEncryption(r registry.Registry) { // to test the correctness of encryption at rest. runEncryption := func(ctx context.Context, t test.Test, c cluster.Cluster) { nodes := c.Spec().NodeCount - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, nodes)) // Check that /_status/stores/local endpoint has encryption status. diff --git a/pkg/cmd/roachtest/tests/event_log.go b/pkg/cmd/roachtest/tests/event_log.go index b41f7c427b4d..ec23c58735a9 100644 --- a/pkg/cmd/roachtest/tests/event_log.go +++ b/pkg/cmd/roachtest/tests/event_log.go @@ -31,7 +31,6 @@ func runEventLog(ctx context.Context, t test.Test, c cluster.Cluster) { NodeID roachpb.NodeID } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) // Verify that "node joined" and "node restart" events are recorded whenever diff --git a/pkg/cmd/roachtest/tests/failover.go b/pkg/cmd/roachtest/tests/failover.go index c8e269ffbab0..dcf268d82b9e 100644 --- a/pkg/cmd/roachtest/tests/failover.go +++ b/pkg/cmd/roachtest/tests/failover.go @@ -227,7 +227,6 @@ func runFailoverChaos(ctx context.Context, t test.Test, c cluster.Cluster, readO failers = append(failers, failer) } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 9)) conn := c.Conn(ctx, t.L(), 1) @@ -401,7 +400,6 @@ func runFailoverPartialLeaseGateway(ctx context.Context, t test.Test, c cluster. failer.Setup(ctx) defer failer.Cleanup(ctx) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 7)) conn := c.Conn(ctx, t.L(), 1) @@ -538,7 +536,6 @@ func runFailoverPartialLeaseLeader(ctx context.Context, t test.Test, c cluster.C failer.Setup(ctx) defer failer.Cleanup(ctx) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 3)) conn := c.Conn(ctx, t.L(), 1) @@ -670,7 +667,6 @@ func runFailoverPartialLeaseLiveness(ctx context.Context, t test.Test, c cluster failer.Setup(ctx) defer failer.Cleanup(ctx) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 7)) conn := c.Conn(ctx, t.L(), 1) @@ -789,7 +785,6 @@ func runFailoverNonSystem( failer.Setup(ctx) defer failer.Cleanup(ctx) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 6)) conn := c.Conn(ctx, t.L(), 1) @@ -899,7 +894,6 @@ func runFailoverLiveness( failer.Setup(ctx) defer failer.Cleanup(ctx) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 4)) conn := c.Conn(ctx, t.L(), 1) @@ -1015,7 +1009,6 @@ func runFailoverSystemNonLiveness( failer.Setup(ctx) defer failer.Cleanup(ctx) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), opts, settings, c.Range(1, 6)) conn := c.Conn(ctx, t.L(), 1) diff --git a/pkg/cmd/roachtest/tests/flowable.go b/pkg/cmd/roachtest/tests/flowable.go index 25a815dc6749..683e8a0fd3f8 100644 --- a/pkg/cmd/roachtest/tests/flowable.go +++ b/pkg/cmd/roachtest/tests/flowable.go @@ -37,7 +37,6 @@ func registerFlowable(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) t.Status("creating database used by tests") diff --git a/pkg/cmd/roachtest/tests/follower_reads.go b/pkg/cmd/roachtest/tests/follower_reads.go index bcd24256cba1..fda4d5e192a0 100644 --- a/pkg/cmd/roachtest/tests/follower_reads.go +++ b/pkg/cmd/roachtest/tests/follower_reads.go @@ -71,7 +71,6 @@ func registerFollowerReads(r registry.Registry) { if c.Cloud() == spec.GCE && c.Spec().Arch == vm.ArchARM64 { t.Skip("arm64 in GCE is available only in us-central1") } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) topology := topologySpec{ multiRegion: true, diff --git a/pkg/cmd/roachtest/tests/gopg.go b/pkg/cmd/roachtest/tests/gopg.go index f8134b29dfa7..477ad624c05c 100644 --- a/pkg/cmd/roachtest/tests/gopg.go +++ b/pkg/cmd/roachtest/tests/gopg.go @@ -53,7 +53,6 @@ func registerGopg(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) if err != nil { diff --git a/pkg/cmd/roachtest/tests/gorm.go b/pkg/cmd/roachtest/tests/gorm.go index 0525ff6e27a5..0a87333440ee 100644 --- a/pkg/cmd/roachtest/tests/gorm.go +++ b/pkg/cmd/roachtest/tests/gorm.go @@ -36,7 +36,6 @@ func registerGORM(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) if err != nil { diff --git a/pkg/cmd/roachtest/tests/gossip.go b/pkg/cmd/roachtest/tests/gossip.go index d45e8a06f89b..ea3afd04665a 100644 --- a/pkg/cmd/roachtest/tests/gossip.go +++ b/pkg/cmd/roachtest/tests/gossip.go @@ -39,7 +39,6 @@ import ( func registerGossip(r registry.Registry) { runGossipChaos := func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--vmodule=*=1") c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.All()) @@ -278,7 +277,6 @@ func (g *gossipUtil) checkConnectedAndFunctional( } func runGossipPeerings(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) // Repeatedly restart a random node and verify that all of the nodes are @@ -312,7 +310,6 @@ func runGossipPeerings(ctx context.Context, t test.Test, c cluster.Cluster) { } func runGossipRestart(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) // Repeatedly stop and restart a cluster and verify that we can perform basic @@ -337,7 +334,6 @@ func runGossipRestart(ctx context.Context, t test.Test, c cluster.Cluster) { } func runGossipRestartNodeOne(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") // Reduce the scan max idle time to speed up evacuation of node 1. settings := install.MakeClusterSettings(install.NumRacksOption(c.Spec().NodeCount)) settings.Env = append(settings.Env, "COCKROACH_SCAN_MAX_IDLE_TIME=5ms") @@ -492,8 +488,6 @@ SELECT count(replicas) } func runCheckLocalityIPAddress(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") - externalIP, err := c.ExternalIP(ctx, t.L(), c.Range(1, c.Spec().NodeCount)) if err != nil { t.Fatal(err) diff --git a/pkg/cmd/roachtest/tests/hibernate.go b/pkg/cmd/roachtest/tests/hibernate.go index 12785b1a3b35..c5b4302d3e0e 100644 --- a/pkg/cmd/roachtest/tests/hibernate.go +++ b/pkg/cmd/roachtest/tests/hibernate.go @@ -92,7 +92,6 @@ func registerHibernate(r registry.Registry, opt hibernateOptions) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) if opt.dbSetupFunc != nil { diff --git a/pkg/cmd/roachtest/tests/hotspotsplits.go b/pkg/cmd/roachtest/tests/hotspotsplits.go index 0abf210875cc..d700ecfaedc0 100644 --- a/pkg/cmd/roachtest/tests/hotspotsplits.go +++ b/pkg/cmd/roachtest/tests/hotspotsplits.go @@ -34,7 +34,6 @@ func registerHotSpotSplits(r registry.Registry) { roachNodes := c.Range(1, c.Spec().NodeCount-1) appNode := c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", appNode) diff --git a/pkg/cmd/roachtest/tests/import.go b/pkg/cmd/roachtest/tests/import.go index 7612ddd20c1c..f0c6f8e5d63f 100644 --- a/pkg/cmd/roachtest/tests/import.go +++ b/pkg/cmd/roachtest/tests/import.go @@ -96,7 +96,6 @@ func registerImportNodeShutdown(r registry.Registry) { if c.Cloud() != spec.GCE && !c.IsLocal() { t.Skip("uses gs://cockroach-fixtures; see https://github.com/cockroachdb/cockroach/issues/105968") } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) gatewayNode := 2 nodeToShutdown := 3 @@ -116,7 +115,6 @@ func registerImportNodeShutdown(r registry.Registry) { if c.Cloud() != spec.GCE && !c.IsLocal() { t.Skip("uses gs://cockroach-fixtures; see https://github.com/cockroachdb/cockroach/issues/105968") } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) gatewayNode := 2 nodeToShutdown := 2 @@ -130,7 +128,6 @@ func registerImportNodeShutdown(r registry.Registry) { func registerImportTPCC(r registry.Registry) { runImportTPCC := func(ctx context.Context, t test.Test, c cluster.Cluster, testName string, timeout time.Duration, warehouses int) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") t.Status("starting csv servers") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) @@ -241,7 +238,6 @@ func registerImportTPCH(r registry.Registry) { } tick, perfBuf := initBulkJobPerfArtifacts(t.Name(), item.timeout) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) conn := c.Conn(ctx, t.L(), 1) if _, err := conn.Exec(`CREATE DATABASE csv;`); err != nil { @@ -351,7 +347,6 @@ func registerImportDecommissioned(r registry.Registry) { warehouses = 10 } - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") t.Status("starting csv servers") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) diff --git a/pkg/cmd/roachtest/tests/import_cancellation.go b/pkg/cmd/roachtest/tests/import_cancellation.go index 77393b80ff6d..9082173a6757 100644 --- a/pkg/cmd/roachtest/tests/import_cancellation.go +++ b/pkg/cmd/roachtest/tests/import_cancellation.go @@ -48,7 +48,6 @@ func registerImportCancellation(r registry.Registry) { } func runImportCancellation(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") // required for tpch startOpts := maybeUseMemoryBudget(t, 50) startOpts.RoachprodOpts.ScheduleBackups = true diff --git a/pkg/cmd/roachtest/tests/inconsistency.go b/pkg/cmd/roachtest/tests/inconsistency.go index 2d1677d01936..477fb1db606d 100644 --- a/pkg/cmd/roachtest/tests/inconsistency.go +++ b/pkg/cmd/roachtest/tests/inconsistency.go @@ -36,7 +36,6 @@ func registerInconsistency(r registry.Registry) { func runInconsistency(ctx context.Context, t test.Test, c cluster.Cluster) { nodes := c.Range(1, 3) - c.Put(ctx, t.Cockroach(), "./cockroach", nodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), nodes) { diff --git a/pkg/cmd/roachtest/tests/indexes.go b/pkg/cmd/roachtest/tests/indexes.go index 6e16ade465ad..5bdaebed30d5 100644 --- a/pkg/cmd/roachtest/tests/indexes.go +++ b/pkg/cmd/roachtest/tests/indexes.go @@ -54,7 +54,6 @@ func registerNIndexes(r registry.Registry, secondaryIndexes int) { gatewayNodes := c.Range(1, nodes/3) loadNode := c.Node(nodes + 1) - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", loadNode) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes) conn := c.Conn(ctx, t.L(), 1) diff --git a/pkg/cmd/roachtest/tests/inverted_index.go b/pkg/cmd/roachtest/tests/inverted_index.go index fbded79e7811..bc0f7ced41ba 100644 --- a/pkg/cmd/roachtest/tests/inverted_index.go +++ b/pkg/cmd/roachtest/tests/inverted_index.go @@ -43,7 +43,6 @@ func runSchemaChangeInvertedIndex(ctx context.Context, t test.Test, c cluster.Cl crdbNodes := c.Range(1, c.Spec().NodeCount-1) workloadNode := c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", workloadNode) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), crdbNodes) diff --git a/pkg/cmd/roachtest/tests/jasyncsql.go b/pkg/cmd/roachtest/tests/jasyncsql.go index f4810bd55e86..fef73f97e4ae 100644 --- a/pkg/cmd/roachtest/tests/jasyncsql.go +++ b/pkg/cmd/roachtest/tests/jasyncsql.go @@ -30,7 +30,6 @@ func registerJasyncSQL(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/jepsen.go b/pkg/cmd/roachtest/tests/jepsen.go index d9b1d6816198..c3d462eec07c 100644 --- a/pkg/cmd/roachtest/tests/jepsen.go +++ b/pkg/cmd/roachtest/tests/jepsen.go @@ -147,13 +147,10 @@ func initJepsen(ctx context.Context, t test.Test, c cluster.Cluster, j jepsenCon c.Run(ctx, c.All(), "sh", "-c", `"sudo apt-get -y update > logs/apt-upgrade.log 2>&1"`) c.Run(ctx, c.All(), "sh", "-c", `"sudo DEBIAN_FRONTEND=noninteractive apt-get -y upgrade -o Dpkg::Options::='--force-confold' -o DPkg::options::='--force-confdef' > logs/apt-upgrade.log 2>&1"`) - // Install the binary on all nodes and package it as jepsen expects. // TODO(bdarnell): copying the raw binary and compressing it on the // other side is silly, but this lets us avoid platform-specific // quirks in tar. The --transform option is only available on gnu // tar. To be able to run from a macOS host with BSD tar we'd need - // use the similar -s option on that platform. - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) // Jepsen expects a tarball that expands to cockroach/cockroach // (which is not how our official builds are laid out). c.Run(ctx, c.All(), "tar --transform s,^,cockroach/, -c -z -f cockroach.tgz cockroach") diff --git a/pkg/cmd/roachtest/tests/knex.go b/pkg/cmd/roachtest/tests/knex.go index 037c1ef9f3a0..5d4167da0425 100644 --- a/pkg/cmd/roachtest/tests/knex.go +++ b/pkg/cmd/roachtest/tests/knex.go @@ -40,7 +40,6 @@ func registerKnex(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/kv.go b/pkg/cmd/roachtest/tests/kv.go index af629be226f0..8558e8b983d1 100644 --- a/pkg/cmd/roachtest/tests/kv.go +++ b/pkg/cmd/roachtest/tests/kv.go @@ -99,7 +99,6 @@ func registerKV(r registry.Registry) { } runKV := func(ctx context.Context, t test.Test, c cluster.Cluster, opts kvOptions) { nodes := c.Spec().NodeCount - 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) // Don't start a scheduled backup on this perf sensitive roachtest that reports to roachperf. @@ -374,7 +373,6 @@ func registerKVContention(r registry.Registry) { Suites: registry.Suites(registry.Nightly), Leases: registry.MetamorphicLeases, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) // Start the cluster with an extremely high txn liveness threshold. @@ -448,7 +446,6 @@ func registerKVQuiescenceDead(r registry.Registry) { SkipPostValidations: registry.PostValidationNoDeadNodes, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { nodes := c.Spec().NodeCount - 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) settings := install.MakeClusterSettings(install.ClusterSettingsOption{ "sql.stats.automatic_collection.enabled": "false", @@ -527,7 +524,6 @@ func registerKVGracefulDraining(r registry.Registry) { Suites: registry.Suites(registry.Nightly), Leases: registry.MetamorphicLeases, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, c.Spec().NodeCount)) nodes := c.Spec().NodeCount - 1 t.Status("starting cluster") @@ -693,7 +689,6 @@ func registerKVSplits(r registry.Registry) { Leases: item.leases, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { nodes := c.Spec().NodeCount - 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) settings := install.MakeClusterSettings() @@ -727,7 +722,6 @@ func registerKVScalability(r registry.Registry) { runScalability := func(ctx context.Context, t test.Test, c cluster.Cluster, percent int) { nodes := c.Spec().NodeCount - 1 - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) const maxPerNodeConcurrency = 64 @@ -785,7 +779,6 @@ func registerKVRangeLookups(r registry.Registry) { nodes := c.Spec().NodeCount - 1 doneInit := make(chan struct{}) doneWorkload := make(chan struct{}) - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, nodes)) @@ -968,7 +961,6 @@ func registerKVRestartImpact(r registry.Registry) { Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { nodes := c.Spec().NodeCount - 1 workloadNode := c.Spec().NodeCount - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--vmodule=store_rebalancer=5,allocator=5,allocator_scorer=5,replicate_queue=5") diff --git a/pkg/cmd/roachtest/tests/kvbench.go b/pkg/cmd/roachtest/tests/kvbench.go index 9afe8dd81c16..31ce5eb3e3fe 100644 --- a/pkg/cmd/roachtest/tests/kvbench.go +++ b/pkg/cmd/roachtest/tests/kvbench.go @@ -205,9 +205,6 @@ func runKVBench(ctx context.Context, t test.Test, c cluster.Cluster, b kvBenchSp roachNodes := loadGrp.roachNodes loadNodes := loadGrp.loadNodes - if err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", roachNodes); err != nil { - t.Fatal(err) - } if err := c.PutE(ctx, t.L(), t.DeprecatedWorkload(), "./workload", loadNodes); err != nil { t.Fatal(err) } diff --git a/pkg/cmd/roachtest/tests/lease_preferences.go b/pkg/cmd/roachtest/tests/lease_preferences.go index 83d8965870b8..c940577ce1e2 100644 --- a/pkg/cmd/roachtest/tests/lease_preferences.go +++ b/pkg/cmd/roachtest/tests/lease_preferences.go @@ -213,7 +213,6 @@ func runLeasePreferences( spec.ranges = 100 } - c.Put(ctx, t.Cockroach(), "./cockroach") t.Status("starting cluster") startNodes(allNodes...) diff --git a/pkg/cmd/roachtest/tests/ledger.go b/pkg/cmd/roachtest/tests/ledger.go index b24c5e57849f..a78387ca7b66 100644 --- a/pkg/cmd/roachtest/tests/ledger.go +++ b/pkg/cmd/roachtest/tests/ledger.go @@ -39,7 +39,6 @@ func registerLedger(r registry.Registry) { gatewayNodes := c.Range(1, nodes/3) loadNode := c.Node(nodes + 1) - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", loadNode) // Don't start a scheduled backup on this perf sensitive roachtest that reports to roachperf. diff --git a/pkg/cmd/roachtest/tests/libpq.go b/pkg/cmd/roachtest/tests/libpq.go index a58a48858bcb..8dc3b55fb8af 100644 --- a/pkg/cmd/roachtest/tests/libpq.go +++ b/pkg/cmd/roachtest/tests/libpq.go @@ -37,7 +37,6 @@ func registerLibPQ(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/liquibase.go b/pkg/cmd/roachtest/tests/liquibase.go index 8c012de3f6a8..5c24150927d0 100644 --- a/pkg/cmd/roachtest/tests/liquibase.go +++ b/pkg/cmd/roachtest/tests/liquibase.go @@ -35,7 +35,6 @@ func registerLiquibase(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/loss_of_quorum_recovery.go b/pkg/cmd/roachtest/tests/loss_of_quorum_recovery.go index e2183615973e..25c16c102bb4 100644 --- a/pkg/cmd/roachtest/tests/loss_of_quorum_recovery.go +++ b/pkg/cmd/roachtest/tests/loss_of_quorum_recovery.go @@ -172,7 +172,6 @@ func runRecoverLossOfQuorum(ctx context.Context, t test.Test, c cluster.Cluster, dbName := "test_db" workloadHistogramFile := "restored.json" - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) settings := install.MakeClusterSettings(install.EnvOption([]string{ "COCKROACH_MIN_RANGE_MAX_BYTES=1", })) @@ -390,7 +389,6 @@ func runHalfOnlineRecoverLossOfQuorum( dbName := "test_db" workloadHistogramFile := "restored.json" - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) settings := install.MakeClusterSettings(install.EnvOption([]string{ "COCKROACH_MIN_RANGE_MAX_BYTES=1", })) diff --git a/pkg/cmd/roachtest/tests/many_splits.go b/pkg/cmd/roachtest/tests/many_splits.go index 46466fab4ecb..e1d18a405b2d 100644 --- a/pkg/cmd/roachtest/tests/many_splits.go +++ b/pkg/cmd/roachtest/tests/many_splits.go @@ -24,7 +24,6 @@ import ( // runManySplits attempts to create 2000 tiny ranges on a 4-node cluster using // left-to-right splits and check the cluster is still live afterwards. func runManySplits(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") settings := install.MakeClusterSettings() settings.Env = append(settings.Env, "COCKROACH_SCAN_MAX_IDLE_TIME=5ms") c.Start(ctx, t.L(), option.DefaultStartOpts(), settings) diff --git a/pkg/cmd/roachtest/tests/mixed_version_cdc.go b/pkg/cmd/roachtest/tests/mixed_version_cdc.go index 9a250bd45de4..5c2f2e8353bc 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_cdc.go +++ b/pkg/cmd/roachtest/tests/mixed_version_cdc.go @@ -131,7 +131,6 @@ func newCDCMixedVersionTester( crdbNodes := c.Range(1, c.Spec().NodeCount-1) lastNode := c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach", lastNode) c.Put(ctx, t.DeprecatedWorkload(), "./workload", lastNode) return cdcMixedVersionTester{ diff --git a/pkg/cmd/roachtest/tests/multitenant.go b/pkg/cmd/roachtest/tests/multitenant.go index 1017fab83b87..bcdf06c1c723 100644 --- a/pkg/cmd/roachtest/tests/multitenant.go +++ b/pkg/cmd/roachtest/tests/multitenant.go @@ -23,8 +23,6 @@ import ( ) func runAcceptanceMultitenant(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(install.SecureOption(true)), c.All()) const tenantID = 123 diff --git a/pkg/cmd/roachtest/tests/multitenant_distsql.go b/pkg/cmd/roachtest/tests/multitenant_distsql.go index af24cc4fd19e..658607388af7 100644 --- a/pkg/cmd/roachtest/tests/multitenant_distsql.go +++ b/pkg/cmd/roachtest/tests/multitenant_distsql.go @@ -59,7 +59,6 @@ func runMultiTenantDistSQL( bundle bool, timeoutMillis int, ) { - c.Put(ctx, t.Cockroach(), "./cockroach") // This test sets a smaller default range size than the default due to // performance and resource limitations. We set the minimum range max bytes to // 1 byte to bypass the guardrails. diff --git a/pkg/cmd/roachtest/tests/multitenant_shared_process.go b/pkg/cmd/roachtest/tests/multitenant_shared_process.go index afdc4c1486d6..5d33aeb82f40 100644 --- a/pkg/cmd/roachtest/tests/multitenant_shared_process.go +++ b/pkg/cmd/roachtest/tests/multitenant_shared_process.go @@ -41,7 +41,6 @@ func registerMultiTenantSharedProcess(r registry.Registry) { workloadNode = c.Node(crdbNodeCount + 1) ) t.Status(`set up Unified Architecture Cluster`) - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", workloadNode) // In order to observe the app tenant's db console, create a secure diff --git a/pkg/cmd/roachtest/tests/multitenant_tpch.go b/pkg/cmd/roachtest/tests/multitenant_tpch.go index 606ce20284a2..89783616522f 100644 --- a/pkg/cmd/roachtest/tests/multitenant_tpch.go +++ b/pkg/cmd/roachtest/tests/multitenant_tpch.go @@ -31,7 +31,6 @@ func runMultiTenantTPCH( ctx context.Context, t test.Test, c cluster.Cluster, enableDirectScans bool, sharedProcess bool, ) { secure := true - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(1)) start := func() { c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(install.SecureOption(secure)), c.All()) diff --git a/pkg/cmd/roachtest/tests/mvcc_gc.go b/pkg/cmd/roachtest/tests/mvcc_gc.go index bc7281e88695..19e56d8b514d 100644 --- a/pkg/cmd/roachtest/tests/mvcc_gc.go +++ b/pkg/cmd/roachtest/tests/mvcc_gc.go @@ -87,7 +87,6 @@ func runMVCCGC(ctx context.Context, t test.Test, c cluster.Cluster) { } c.SetRandomSeed(randomSeed) } - c.Put(ctx, t.Cockroach(), "./cockroach") s := install.MakeClusterSettings() s.Env = append(s.Env, "COCKROACH_SCAN_INTERVAL=30s") // Disable an automatic scheduled backup as it would mess with the gc ttl this test relies on. diff --git a/pkg/cmd/roachtest/tests/network.go b/pkg/cmd/roachtest/tests/network.go index da270a4331e7..331cd4bfe0fd 100644 --- a/pkg/cmd/roachtest/tests/network.go +++ b/pkg/cmd/roachtest/tests/network.go @@ -37,8 +37,6 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste n := c.Spec().NodeCount serverNodes, clientNode := c.Range(1, n-1), c.Node(n) - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) - t.L().Printf("starting nodes to initialize TLS certs...") // NB: we need to start two times, because when we use // c.Start() separately on nodes 1 and nodes 2-3, diff --git a/pkg/cmd/roachtest/tests/network_logging.go b/pkg/cmd/roachtest/tests/network_logging.go index 01d41bf05e21..a1aeaedddc0e 100644 --- a/pkg/cmd/roachtest/tests/network_logging.go +++ b/pkg/cmd/roachtest/tests/network_logging.go @@ -61,7 +61,6 @@ func registerNetworkLogging(r registry.Registry) { // Install Cockroach, including on the workload node, // since we'll use ./cockroach workload. t.Status("installing cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) // Start each node with a log config containing fluent-server and http-server sinks. t.Status("starting cockroach on nodes") diff --git a/pkg/cmd/roachtest/tests/nodejs_postgres.go b/pkg/cmd/roachtest/tests/nodejs_postgres.go index 419b804f8c1c..d06e105290cf 100644 --- a/pkg/cmd/roachtest/tests/nodejs_postgres.go +++ b/pkg/cmd/roachtest/tests/nodejs_postgres.go @@ -42,10 +42,8 @@ func registerNodeJSPostgres(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", c.All()) - require.NoError(t, err) settings := install.MakeClusterSettings(install.SecureOption(true)) - err = c.StartE(ctx, t.L(), option.DefaultStartOptsInMemory(), settings) + err := c.StartE(ctx, t.L(), option.DefaultStartOptsInMemory(), settings) require.NoError(t, err) const user = "testuser" // certs auto-generated by roachprod start --secure diff --git a/pkg/cmd/roachtest/tests/npgsql.go b/pkg/cmd/roachtest/tests/npgsql.go index 1c6041c86194..219333b3e8d2 100644 --- a/pkg/cmd/roachtest/tests/npgsql.go +++ b/pkg/cmd/roachtest/tests/npgsql.go @@ -42,7 +42,6 @@ func registerNpgsql(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(install.SecureOption(true)), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/pgjdbc.go b/pkg/cmd/roachtest/tests/pgjdbc.go index 936407c55dd3..0781f5c676ec 100644 --- a/pkg/cmd/roachtest/tests/pgjdbc.go +++ b/pkg/cmd/roachtest/tests/pgjdbc.go @@ -41,7 +41,6 @@ func registerPgjdbc(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(install.SecureOption(true)), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/pgx.go b/pkg/cmd/roachtest/tests/pgx.go index 3578b22bea9a..983b5903cf88 100644 --- a/pkg/cmd/roachtest/tests/pgx.go +++ b/pkg/cmd/roachtest/tests/pgx.go @@ -43,7 +43,6 @@ func registerPgx(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/pop.go b/pkg/cmd/roachtest/tests/pop.go index d421b85eeb41..a86f10946c6a 100644 --- a/pkg/cmd/roachtest/tests/pop.go +++ b/pkg/cmd/roachtest/tests/pop.go @@ -33,7 +33,6 @@ func registerPop(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) if err != nil { diff --git a/pkg/cmd/roachtest/tests/process_lock.go b/pkg/cmd/roachtest/tests/process_lock.go index e2ef0efcdd5c..16f4aa479ed8 100644 --- a/pkg/cmd/roachtest/tests/process_lock.go +++ b/pkg/cmd/roachtest/tests/process_lock.go @@ -49,7 +49,6 @@ func registerProcessLock(r registry.Registry) { startSettings.Env = append(startSettings.Env, "COCKROACH_AUTO_BALLAST=false") t.Status("starting cluster") - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), startOpts, startSettings, c.Range(1, 3)) // Wait for upreplication. diff --git a/pkg/cmd/roachtest/tests/psycopg.go b/pkg/cmd/roachtest/tests/psycopg.go index 01eb920d5e90..4a27015976a1 100644 --- a/pkg/cmd/roachtest/tests/psycopg.go +++ b/pkg/cmd/roachtest/tests/psycopg.go @@ -39,7 +39,6 @@ func registerPsycopg(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/query_comparison_util.go b/pkg/cmd/roachtest/tests/query_comparison_util.go index 82ff3f97c213..2a79cf31f22c 100644 --- a/pkg/cmd/roachtest/tests/query_comparison_util.go +++ b/pkg/cmd/roachtest/tests/query_comparison_util.go @@ -94,8 +94,6 @@ func runQueryComparison( } } - c.Put(clusterCtx, t.Cockroach(), "./cockroach") - for i := 0; ; i++ { if shouldExit() { return diff --git a/pkg/cmd/roachtest/tests/queue.go b/pkg/cmd/roachtest/tests/queue.go index 7521b9173d86..4b53b13a8d3f 100644 --- a/pkg/cmd/roachtest/tests/queue.go +++ b/pkg/cmd/roachtest/tests/queue.go @@ -46,7 +46,6 @@ func runQueue(ctx context.Context, t test.Test, c cluster.Cluster) { workloadNode := c.Spec().NodeCount // Distribute programs to the correct nodes and start CockroachDB. - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, dbNodeCount)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workloadNode)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, dbNodeCount)) diff --git a/pkg/cmd/roachtest/tests/quit.go b/pkg/cmd/roachtest/tests/quit.go index 126450a52e72..cb70a6ad7d96 100644 --- a/pkg/cmd/roachtest/tests/quit.go +++ b/pkg/cmd/roachtest/tests/quit.go @@ -57,7 +57,6 @@ func runQuitTransfersLeases( func (q *quitTest) init(ctx context.Context) { q.args = []string{"--vmodule=replica_proposal=1,allocator=3,allocator_scorer=3"} q.env = []string{"COCKROACH_SCAN_MAX_IDLE_TIME=5ms"} - q.c.Put(ctx, q.t.Cockroach(), "./cockroach") settings := install.MakeClusterSettings(install.EnvOption(q.env)) startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, q.args...) diff --git a/pkg/cmd/roachtest/tests/rapid_restart.go b/pkg/cmd/roachtest/tests/rapid_restart.go index 52d513c2bade..c3cfcdd5bb03 100644 --- a/pkg/cmd/roachtest/tests/rapid_restart.go +++ b/pkg/cmd/roachtest/tests/rapid_restart.go @@ -27,7 +27,6 @@ import ( func runRapidRestart(ctx context.Context, t test.Test, c cluster.Cluster) { // Use a single-node cluster which speeds the stop/start cycle. node := c.Node(1) - c.Put(ctx, t.Cockroach(), "./cockroach", node) // In a loop, bootstrap a new single-node cluster and immediately kill // it. This is more effective at finding problems than restarting an existing diff --git a/pkg/cmd/roachtest/tests/rebalance_load.go b/pkg/cmd/roachtest/tests/rebalance_load.go index 95c15aa4e5a3..2e6cf4aa4523 100644 --- a/pkg/cmd/roachtest/tests/rebalance_load.go +++ b/pkg/cmd/roachtest/tests/rebalance_load.go @@ -109,7 +109,6 @@ func registerRebalanceLoad(r registry.Registry) { c.Start(ctx, t.L(), startOpts, settings, roachNodes) upgradeNodes(ctx, t, c, nodesToUpgrade, startOpts, clusterupgrade.CurrentVersion()) } else { - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Start(ctx, t.L(), startOpts, settings, roachNodes) } diff --git a/pkg/cmd/roachtest/tests/replicagc.go b/pkg/cmd/roachtest/tests/replicagc.go index 023adffff9d7..c621c3970c47 100644 --- a/pkg/cmd/roachtest/tests/replicagc.go +++ b/pkg/cmd/roachtest/tests/replicagc.go @@ -65,7 +65,6 @@ func runReplicaGCChangedPeers( t.Fatal("test needs to be run with 6 nodes") } - c.Put(ctx, t.Cockroach(), "./cockroach") settings := install.MakeClusterSettings(install.EnvOption([]string{"COCKROACH_SCAN_MAX_IDLE_TIME=5ms"})) c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Range(1, 3)) diff --git a/pkg/cmd/roachtest/tests/restart.go b/pkg/cmd/roachtest/tests/restart.go index 2975e8beece6..e95c39e87d82 100644 --- a/pkg/cmd/roachtest/tests/restart.go +++ b/pkg/cmd/roachtest/tests/restart.go @@ -29,7 +29,6 @@ func runRestart(ctx context.Context, t test.Test, c cluster.Cluster, downDuratio const restartNode = 3 t.Status("installing cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--vmodule=raft_log_queue=3") c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), crdbNodes) diff --git a/pkg/cmd/roachtest/tests/restore.go b/pkg/cmd/roachtest/tests/restore.go index 1060bb15a105..b0422235dfab 100644 --- a/pkg/cmd/roachtest/tests/restore.go +++ b/pkg/cmd/roachtest/tests/restore.go @@ -892,7 +892,6 @@ func (rd *restoreDriver) prepareCluster(ctx context.Context) { // For now, only run the test on the cloud provider that also stores the backup. rd.t.Skipf("test configured to run on %s", rd.sp.backup.cloud) } - rd.c.Put(ctx, rd.t.Cockroach(), "./cockroach") rd.c.Start(ctx, rd.t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings()) rd.getAOST(ctx) } diff --git a/pkg/cmd/roachtest/tests/roachmart.go b/pkg/cmd/roachtest/tests/roachmart.go index c938ee474f0a..1a7c6b891d5f 100644 --- a/pkg/cmd/roachtest/tests/roachmart.go +++ b/pkg/cmd/roachtest/tests/roachmart.go @@ -24,7 +24,6 @@ import ( func registerRoachmart(r registry.Registry) { runRoachmart := func(ctx context.Context, t test.Test, c cluster.Cluster, partition bool) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) diff --git a/pkg/cmd/roachtest/tests/roachtest.go b/pkg/cmd/roachtest/tests/roachtest.go index 9868ce0ec074..1522c38ced47 100644 --- a/pkg/cmd/roachtest/tests/roachtest.go +++ b/pkg/cmd/roachtest/tests/roachtest.go @@ -55,7 +55,6 @@ func registerRoachtest(r registry.Registry) { Owner: registry.OwnerTestEng, Run: func(_ context.Context, t test.Test, c cluster.Cluster) { ctx := context.Background() // intentional - c.Put(ctx, t.Cockroach(), "cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) time.Sleep(time.Hour) }, diff --git a/pkg/cmd/roachtest/tests/ruby_pg.go b/pkg/cmd/roachtest/tests/ruby_pg.go index c44d58381445..f2fbfd9a15b5 100644 --- a/pkg/cmd/roachtest/tests/ruby_pg.go +++ b/pkg/cmd/roachtest/tests/ruby_pg.go @@ -51,7 +51,6 @@ func registerRubyPG(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/rust_postgres.go b/pkg/cmd/roachtest/tests/rust_postgres.go index 73ea7c899592..53fc97db9776 100644 --- a/pkg/cmd/roachtest/tests/rust_postgres.go +++ b/pkg/cmd/roachtest/tests/rust_postgres.go @@ -31,7 +31,6 @@ func registerRustPostgres(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) // Most other ORM tests use an in-memory cluster. However, for this test, we // need to restart the cluster with a different port, so we need disk diff --git a/pkg/cmd/roachtest/tests/schemachange.go b/pkg/cmd/roachtest/tests/schemachange.go index a501b1a40ce5..f659551ecb2c 100644 --- a/pkg/cmd/roachtest/tests/schemachange.go +++ b/pkg/cmd/roachtest/tests/schemachange.go @@ -41,7 +41,6 @@ func registerSchemaChangeDuringKV(r registry.Registry) { } const fixturePath = `gs://cockroach-fixtures/workload/tpch/scalefactor=10/backup?AUTH=implicit` - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) @@ -372,7 +371,6 @@ func makeSchemaChangeBulkIngestTest( crdbNodes := c.Range(1, c.Spec().NodeCount-1) workloadNode := c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Put(ctx, t.DeprecatedWorkload(), "./workload", workloadNode) // TODO (lucy): Remove flag once the faster import is enabled by default settings := install.MakeClusterSettings(install.EnvOption([]string{"COCKROACH_IMPORT_WORKLOAD_FASTER=true"})) diff --git a/pkg/cmd/roachtest/tests/schemachange_random_load.go b/pkg/cmd/roachtest/tests/schemachange_random_load.go index 583948b78a7a..5f75bc5e2139 100644 --- a/pkg/cmd/roachtest/tests/schemachange_random_load.go +++ b/pkg/cmd/roachtest/tests/schemachange_random_load.go @@ -93,7 +93,6 @@ func runSchemaChangeRandomLoad( loadNode := c.Node(1) roachNodes := c.Range(1, c.Spec().NodeCount) t.Status("copying binaries") - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", loadNode) t.Status("starting cockroach nodes") diff --git a/pkg/cmd/roachtest/tests/sequelize.go b/pkg/cmd/roachtest/tests/sequelize.go index 6647fb6d902b..9a6c11400a1c 100644 --- a/pkg/cmd/roachtest/tests/sequelize.go +++ b/pkg/cmd/roachtest/tests/sequelize.go @@ -41,7 +41,6 @@ func registerSequelize(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/slow_drain.go b/pkg/cmd/roachtest/tests/slow_drain.go index 2003b4aca0aa..17867bb22d55 100644 --- a/pkg/cmd/roachtest/tests/slow_drain.go +++ b/pkg/cmd/roachtest/tests/slow_drain.go @@ -57,13 +57,10 @@ func runSlowDrain(ctx context.Context, t test.Test, c cluster.Cluster, duration var verboseStoreLogRe = "failed to transfer lease.*when draining.*no suitable transfer target found" - err := c.PutE(ctx, t.L(), t.Cockroach(), "./cockroach", c.All()) - require.NoError(t, err) - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.All()) run := func(db *gosql.DB, stmt string) { - _, err = db.ExecContext(ctx, stmt) + _, err := db.ExecContext(ctx, stmt) require.NoError(t, err) t.L().Printf("run: %s\n", stmt) @@ -151,6 +148,6 @@ func runSlowDrain(ctx context.Context, t test.Test, c cluster.Cluster, duration // Expect the drain timeout to expire. t.Status("waiting for the drain timeout to elapse...") - err = m.WaitE() + err := m.WaitE() require.Error(t, err) } diff --git a/pkg/cmd/roachtest/tests/smoketest_secure.go b/pkg/cmd/roachtest/tests/smoketest_secure.go index a4b5da81e8e3..6100921fa538 100644 --- a/pkg/cmd/roachtest/tests/smoketest_secure.go +++ b/pkg/cmd/roachtest/tests/smoketest_secure.go @@ -35,7 +35,6 @@ func registerSecure(r registry.Registry) { Cluster: r.MakeClusterSpec(numNodes), Leases: registry.MetamorphicLeases, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") settings := install.MakeClusterSettings(install.SecureOption(true)) c.Start(ctx, t.L(), option.DefaultStartOpts(), settings) db := c.Conn(ctx, t.L(), 1) @@ -59,7 +58,6 @@ func registerSecure(r registry.Registry) { // multitenantSmokeTest verifies that a secure sql pod can connect to kv server // and that tenant is is properly transmitted via cert. func multitenantSmokeTest(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") settings := install.MakeClusterSettings(install.SecureOption(true)) c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Node(1)) diff --git a/pkg/cmd/roachtest/tests/split.go b/pkg/cmd/roachtest/tests/split.go index c3d49b287094..f039e8034104 100644 --- a/pkg/cmd/roachtest/tests/split.go +++ b/pkg/cmd/roachtest/tests/split.go @@ -427,7 +427,6 @@ func runLoadSplits(ctx context.Context, t test.Test, c cluster.Cluster, params s crdbNodes := c.Range(1, c.Spec().NodeCount-1) workloadNode := c.Node(c.Spec().NodeCount) - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", workloadNode) // We run this without metamorphic constants as the tests make // incorrect assumptions about the absolute values of QPS. @@ -550,6 +549,9 @@ func registerLargeRange(r registry.Registry) { Suites: registry.Suites(registry.Nightly), Leases: registry.MetamorphicLeases, Timeout: 5 * time.Hour, + // Never run with runtime assertions as this makes this test take + // too long to complete. + CockroachBinary: registry.StandardCockroach, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { runLargeRangeSplits(ctx, t, c, size) }, @@ -587,9 +589,6 @@ func runLargeRangeSplits(ctx context.Context, t test.Test, c cluster.Cluster, si rows := size / rowEstimate const minBytes = 16 << 20 // 16 MB - // Never run with runtime assertions as this makes this test take - // too long to complete. - c.Put(ctx, t.StandardCockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.All()) numNodes := c.Spec().NodeCount c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Node(1)) diff --git a/pkg/cmd/roachtest/tests/sqlalchemy.go b/pkg/cmd/roachtest/tests/sqlalchemy.go index ee5d09053ae9..e9b1aa63396e 100644 --- a/pkg/cmd/roachtest/tests/sqlalchemy.go +++ b/pkg/cmd/roachtest/tests/sqlalchemy.go @@ -126,7 +126,6 @@ func runSQLAlchemy(ctx context.Context, t test.Test, c cluster.Cluster) { // Phew, after having setup all that, let's actually run the test. t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) version, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/sqlsmith.go b/pkg/cmd/roachtest/tests/sqlsmith.go index a6a6580cc8cc..f6f57b1479be 100644 --- a/pkg/cmd/roachtest/tests/sqlsmith.go +++ b/pkg/cmd/roachtest/tests/sqlsmith.go @@ -101,7 +101,6 @@ WITH into_db = 'defaultdb', unsafe_restore_incompatible_version; t.L().Printf("seed: %d", seed) c.SetRandomSeed(rng.Int63()) - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) setupFunc, ok := setups[setupName] diff --git a/pkg/cmd/roachtest/tests/sstable_corruption.go b/pkg/cmd/roachtest/tests/sstable_corruption.go index feb442036ed1..160ac94cabd2 100644 --- a/pkg/cmd/roachtest/tests/sstable_corruption.go +++ b/pkg/cmd/roachtest/tests/sstable_corruption.go @@ -32,7 +32,6 @@ func runSSTableCorruption(ctx context.Context, t test.Test, c cluster.Cluster) { corruptNodes := crdbNodes t.Status("installing cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", crdbNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), crdbNodes) { diff --git a/pkg/cmd/roachtest/tests/status_server.go b/pkg/cmd/roachtest/tests/status_server.go index 347574dd6654..c513848e371d 100644 --- a/pkg/cmd/roachtest/tests/status_server.go +++ b/pkg/cmd/roachtest/tests/status_server.go @@ -28,7 +28,6 @@ import ( ) func runStatusServer(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) // Get the ids for each node. diff --git a/pkg/cmd/roachtest/tests/sysbench.go b/pkg/cmd/roachtest/tests/sysbench.go index fceae814796b..556aa0a20c3d 100644 --- a/pkg/cmd/roachtest/tests/sysbench.go +++ b/pkg/cmd/roachtest/tests/sysbench.go @@ -97,7 +97,6 @@ func runSysbench(ctx context.Context, t test.Test, c cluster.Cluster, opts sysbe loadNode := c.Node(c.Spec().NodeCount) t.Status("installing cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", allNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes) err := WaitFor3XReplication(ctx, t, c.Conn(ctx, t.L(), allNodes[0])) require.NoError(t, err) diff --git a/pkg/cmd/roachtest/tests/tlp.go b/pkg/cmd/roachtest/tests/tlp.go index a34dbe853338..b96ed6722f7f 100644 --- a/pkg/cmd/roachtest/tests/tlp.go +++ b/pkg/cmd/roachtest/tests/tlp.go @@ -66,8 +66,6 @@ func runTLP(ctx context.Context, t test.Test, c cluster.Cluster) { } } - c.Put(ctx, t.Cockroach(), "./cockroach") - for i := 0; ; i++ { c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) if shouldExit() { diff --git a/pkg/cmd/roachtest/tests/tombstones.go b/pkg/cmd/roachtest/tests/tombstones.go index dc72407ea224..38fc01aa0155 100644 --- a/pkg/cmd/roachtest/tests/tombstones.go +++ b/pkg/cmd/roachtest/tests/tombstones.go @@ -47,7 +47,6 @@ func registerPointTombstone(r registry.Registry) { startSettings.Env = append(startSettings.Env, "COCKROACH_AUTO_BALLAST=false") t.Status("starting cluster") - c.Put(ctx, t.Cockroach(), "./cockroach") c.Start(ctx, t.L(), startOpts, startSettings, c.Range(1, 3)) // Wait for upreplication. diff --git a/pkg/cmd/roachtest/tests/tpcc.go b/pkg/cmd/roachtest/tests/tpcc.go index 3228eeb34353..963135c227ad 100644 --- a/pkg/cmd/roachtest/tests/tpcc.go +++ b/pkg/cmd/roachtest/tests/tpcc.go @@ -145,9 +145,6 @@ func setupTPCC( if opts.Start == nil { opts.Start = func(ctx context.Context, t test.Test, c cluster.Cluster) { - // NB: workloadNode also needs ./cockroach because - // of `./cockroach workload` for usingImport. - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) settings := install.MakeClusterSettings() if c.IsLocal() { settings.Env = append(settings.Env, "COCKROACH_SCAN_INTERVAL=200ms") @@ -1422,11 +1419,6 @@ func runTPCCBench(ctx context.Context, t test.Test, c cluster.Cluster, b tpccBen loadGroups := makeLoadGroups(c, numZones, b.Nodes, numLoadGroups) roachNodes := loadGroups.roachNodes() loadNodes := loadGroups.loadNodes() - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) - // Fixture import needs './cockroach workload' on loadNodes[0], - // and if we use haproxy (see below) we need it on the others - // as well. - c.Put(ctx, t.Cockroach(), "./cockroach", loadNodes) // Don't encrypt in tpccbench tests. startOpts, settings := b.startOpts() c.Start(ctx, t.L(), startOpts, settings, roachNodes) diff --git a/pkg/cmd/roachtest/tests/tpcdsvec.go b/pkg/cmd/roachtest/tests/tpcdsvec.go index c0871d839b83..fbf35e918b34 100644 --- a/pkg/cmd/roachtest/tests/tpcdsvec.go +++ b/pkg/cmd/roachtest/tests/tpcdsvec.go @@ -59,7 +59,6 @@ func registerTPCDSVec(r registry.Registry) { } runTPCDSVec := func(ctx context.Context, t test.Test, c cluster.Cluster) { - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) clusterConn := c.Conn(ctx, t.L(), 1) diff --git a/pkg/cmd/roachtest/tests/tpce.go b/pkg/cmd/roachtest/tests/tpce.go index f970da6fddc4..db69d0512c3b 100644 --- a/pkg/cmd/roachtest/tests/tpce.go +++ b/pkg/cmd/roachtest/tests/tpce.go @@ -145,10 +145,6 @@ func runTPCE(ctx context.Context, t test.Test, c cluster.Cluster, opts tpceOptio if opts.start == nil { opts.start = func(ctx context.Context, t test.Test, c cluster.Cluster) { t.Status("installing cockroach") - // Never run with runtime assertions as this makes this test take - // too long to complete. - c.Put(ctx, t.StandardCockroach(), "./cockroach", crdbNodes) - startOpts := option.DefaultStartOpts() startOpts.RoachprodOpts.StoreCount = opts.ssds settings := install.MakeClusterSettings(install.NumRacksOption(racks)) @@ -255,6 +251,9 @@ func registerTPCE(r registry.Registry) { Cluster: r.MakeClusterSpec(smallNightly.nodes+1, spec.CPU(smallNightly.cpus), spec.SSD(smallNightly.ssds)), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), + // Never run with runtime assertions as this makes this test take + // too long to complete. + CockroachBinary: registry.StandardCockroach, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { runTPCE(ctx, t, c, smallNightly) }, diff --git a/pkg/cmd/roachtest/tests/tpch_concurrency.go b/pkg/cmd/roachtest/tests/tpch_concurrency.go index a898f74d13f9..ff0b225281dd 100644 --- a/pkg/cmd/roachtest/tests/tpch_concurrency.go +++ b/pkg/cmd/roachtest/tests/tpch_concurrency.go @@ -33,10 +33,6 @@ func registerTPCHConcurrency(r registry.Registry) { c cluster.Cluster, disableStreamer bool, ) { - // We run this test without runtime assertions as it pushes the VMs way - // past the overload point, so it cannot withstand any metamorphic - // perturbations. - c.Put(ctx, t.StandardCockroach(), "./cockroach", c.Range(1, numNodes-1)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(numNodes)) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), c.Range(1, numNodes-1)) @@ -222,6 +218,10 @@ func registerTPCHConcurrency(r registry.Registry) { Cluster: r.MakeClusterSpec(numNodes), CompatibleClouds: registry.AllExceptAWS, Suites: registry.Suites(registry.Nightly), + // We run this test without runtime assertions as it pushes the VMs way + // past the overload point, so it cannot withstand any metamorphic + // perturbations. + CockroachBinary: registry.StandardCockroach, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { runTPCHConcurrency(ctx, t, c, true /* disableStreamer */) }, diff --git a/pkg/cmd/roachtest/tests/tpchbench.go b/pkg/cmd/roachtest/tests/tpchbench.go index 3b7af2429693..80c948c0c07c 100644 --- a/pkg/cmd/roachtest/tests/tpchbench.go +++ b/pkg/cmd/roachtest/tests/tpchbench.go @@ -57,7 +57,6 @@ func runTPCHBench(ctx context.Context, t test.Test, c cluster.Cluster, b tpchBen loadNode := c.Node(c.Spec().NodeCount) t.Status("copying binaries") - c.Put(ctx, t.Cockroach(), "./cockroach", roachNodes) c.Put(ctx, t.DeprecatedWorkload(), "./workload", loadNode) filename := b.benchType diff --git a/pkg/cmd/roachtest/tests/tpchvec.go b/pkg/cmd/roachtest/tests/tpchvec.go index 4c34837a26c2..52b684fad7c6 100644 --- a/pkg/cmd/roachtest/tests/tpchvec.go +++ b/pkg/cmd/roachtest/tests/tpchvec.go @@ -589,7 +589,6 @@ func runTPCHVec( testRun func(ctx context.Context, t test.Test, c cluster.Cluster, conn *gosql.DB, tc tpchVecTestCase), ) { firstNode := c.Node(1) - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", firstNode) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings()) diff --git a/pkg/cmd/roachtest/tests/typeorm.go b/pkg/cmd/roachtest/tests/typeorm.go index c68bd7abc34d..c810612c4830 100644 --- a/pkg/cmd/roachtest/tests/typeorm.go +++ b/pkg/cmd/roachtest/tests/typeorm.go @@ -44,7 +44,6 @@ func registerTypeORM(r registry.Registry) { } node := c.Node(1) t.Status("setting up cockroach") - c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Start(ctx, t.L(), option.DefaultStartOptsInMemory(), install.MakeClusterSettings(), c.All()) cockroachVersion, err := fetchCockroachVersion(ctx, t.L(), c, node[0]) diff --git a/pkg/cmd/roachtest/tests/ycsb.go b/pkg/cmd/roachtest/tests/ycsb.go index 969561374fe0..f61f3d74a8ed 100644 --- a/pkg/cmd/roachtest/tests/ycsb.go +++ b/pkg/cmd/roachtest/tests/ycsb.go @@ -67,7 +67,6 @@ func registerYCSB(r registry.Registry) { settings.Env = append(settings.Env, "COCKROACH_GLOBAL_MVCC_RANGE_TOMBSTONE=true") } - c.Put(ctx, t.Cockroach(), "./cockroach", c.Range(1, nodes)) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(nodes+1)) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), settings, c.Range(1, nodes)) From de67b88b33e3a1412baacf103810a658568e5776 Mon Sep 17 00:00:00 2001 From: DarrylWong Date: Tue, 7 Nov 2023 14:51:12 -0500 Subject: [PATCH 2/3] roachtest: print cockroach random seed to test.log This change prints the cockroach random seed used by metamorphic builds to test.log for ease of debugging. Before this seed was found only in cockroach.log. --- pkg/cmd/roachtest/cluster.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 19d3d922ccfc..3d6cf2d5fe1b 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/tests" "github.com/cockroachdb/cockroach/pkg/roachprod" "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/install" @@ -1756,10 +1757,14 @@ func (c *clusterImpl) PutE( func (c *clusterImpl) PutCockroach(ctx context.Context, l *logger.Logger, t *testImpl) error { switch t.spec.CockroachBinary { case registry.RandomizedCockroach: + if tests.UsingRuntimeAssertions(t) { + t.l.Printf("To reproduce the same set of metamorphic constants, run this test with %s=%d", test.EnvAssertionsEnabledSeed, c.cockroachRandomSeed()) + } return c.PutE(ctx, l, t.Cockroach(), test.DefaultCockroachPath, c.All()) case registry.StandardCockroach: return c.PutE(ctx, l, t.StandardCockroach(), test.DefaultCockroachPath, c.All()) case registry.RuntimeAssertionsCockroach: + t.l.Printf("To reproduce the same set of metamorphic constants, run this test with %s=%d", test.EnvAssertionsEnabledSeed, c.cockroachRandomSeed()) return c.PutE(ctx, l, t.RuntimeAssertionsCockroach(), test.DefaultCockroachPath, c.All()) default: return errors.Errorf("Specified cockroach binary does not exist.") From 509c5700a89be9f0ee947c244cdb669bf8e6206b Mon Sep 17 00:00:00 2001 From: DarrylWong Date: Wed, 8 Nov 2023 10:24:14 -0500 Subject: [PATCH 3/3] roachtest: use cockroach binary test spec for github issue labels Before if a test uploaded a non t.Cockroach() binary github posting would have no knowledge of that and naively use t.Cockroach() to determine metamorphism or not. This fixes that. --- pkg/cmd/roachtest/github.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go index f577c5fb56ca..c7f7e064a91c 100644 --- a/pkg/cmd/roachtest/github.go +++ b/pkg/cmd/roachtest/github.go @@ -259,7 +259,16 @@ func (g *githubIssues) MaybePost(t *testImpl, l *logger.Logger, message string) return nil } - postRequest, err := g.createPostRequest(t.Name(), t.start, t.end, t.spec, t.firstFailure(), message, tests.UsingRuntimeAssertions(t)) + var metamorphicBuild bool + switch t.spec.CockroachBinary { + case registry.StandardCockroach: + metamorphicBuild = false + case registry.RuntimeAssertionsCockroach: + metamorphicBuild = true + default: + metamorphicBuild = tests.UsingRuntimeAssertions(t) + } + postRequest, err := g.createPostRequest(t.Name(), t.start, t.end, t.spec, t.firstFailure(), message, metamorphicBuild) if err != nil { return err }