From 2c8115605de7d167e196f5c20d2c7a052412cfb5 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Fri, 23 Jun 2023 12:18:35 -0400 Subject: [PATCH] roachtest: introduce `preserveCerts` parameter to `Wipe` This parameter already existed in the underlying `cluster_synced` implemenation, but the `cluster` interface exposed to roachtsts forced that argument to `false`. This commit changes the interface to expose the parameter, allowing tests to wipe clusters while keeping the certs intact. Epic: none Release note: None --- pkg/cmd/roachtest/cluster.go | 12 +++++++----- pkg/cmd/roachtest/cluster/cluster_interface.go | 4 ++-- pkg/cmd/roachtest/test_runner.go | 2 +- pkg/cmd/roachtest/tests/allocation_bench.go | 2 +- pkg/cmd/roachtest/tests/autoupgrade.go | 2 +- pkg/cmd/roachtest/tests/clock_jump_crash.go | 2 +- pkg/cmd/roachtest/tests/clock_monotonic.go | 2 +- pkg/cmd/roachtest/tests/cluster_init.go | 2 +- pkg/cmd/roachtest/tests/decommission.go | 4 ++-- pkg/cmd/roachtest/tests/decommission_self.go | 2 +- pkg/cmd/roachtest/tests/inconsistency.go | 2 +- pkg/cmd/roachtest/tests/kv.go | 2 +- pkg/cmd/roachtest/tests/kvbench.go | 2 +- pkg/cmd/roachtest/tests/mixed_version_backup.go | 2 +- pkg/cmd/roachtest/tests/multitenant_tpch.go | 2 +- pkg/cmd/roachtest/tests/query_comparison_util.go | 2 +- pkg/cmd/roachtest/tests/rapid_restart.go | 4 ++-- pkg/cmd/roachtest/tests/sstable_corruption.go | 4 ++-- pkg/cmd/roachtest/tests/tlp.go | 2 +- pkg/cmd/roachtest/tests/tpc_utils.go | 2 +- pkg/cmd/roachtest/tests/tpcc.go | 2 +- .../validate_system_schema_after_version_upgrade.go | 2 +- pkg/cmd/roachtest/tests/versionupgrade.go | 9 +++++++-- 23 files changed, 39 insertions(+), 32 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 5f2dcac22881..1c975e4aadb2 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1201,7 +1201,7 @@ func attachToExistingCluster( } if !opt.skipWipe { if clusterWipe { - if err := c.WipeE(ctx, l, c.All()); err != nil { + if err := c.WipeE(ctx, l, false /* preserveCerts */, c.All()); err != nil { return nil, err } } else { @@ -2246,7 +2246,9 @@ func (c *clusterImpl) Signal( // WipeE wipes a subset of the nodes in a cluster. See cluster.Start() for a // description of the nodes parameter. -func (c *clusterImpl) WipeE(ctx context.Context, l *logger.Logger, nodes ...option.Option) error { +func (c *clusterImpl) WipeE( + ctx context.Context, l *logger.Logger, preserveCerts bool, nodes ...option.Option, +) error { if ctx.Err() != nil { return errors.Wrap(ctx.Err(), "cluster.WipeE") } @@ -2256,16 +2258,16 @@ func (c *clusterImpl) WipeE(ctx context.Context, l *logger.Logger, nodes ...opti } c.setStatusForClusterOpt("wiping", false, nodes...) defer c.clearStatusForClusterOpt(false) - return roachprod.Wipe(ctx, l, c.MakeNodes(nodes...), false /* preserveCerts */) + return roachprod.Wipe(ctx, l, c.MakeNodes(nodes...), preserveCerts) } // Wipe is like WipeE, except instead of returning an error, it does // c.t.Fatal(). c.t needs to be set. -func (c *clusterImpl) Wipe(ctx context.Context, nodes ...option.Option) { +func (c *clusterImpl) Wipe(ctx context.Context, preserveCerts bool, nodes ...option.Option) { if ctx.Err() != nil { return } - if err := c.WipeE(ctx, c.l, nodes...); err != nil { + if err := c.WipeE(ctx, c.l, preserveCerts, nodes...); err != nil { c.t.Fatal(err) } } diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index 62b80862d1cb..fecb5421cbaf 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -115,8 +115,8 @@ type Cluster interface { // Deleting CockroachDB data and logs on nodes. - WipeE(ctx context.Context, l *logger.Logger, opts ...option.Option) error - Wipe(ctx context.Context, opts ...option.Option) + WipeE(ctx context.Context, l *logger.Logger, preserveCerts bool, opts ...option.Option) error + Wipe(ctx context.Context, preserveCerts bool, opts ...option.Option) // Internal niche tools. diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 21e041573bff..c7e87abcbf7f 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -587,7 +587,7 @@ func (r *testRunner) runWorker( if c != nil && testToRun.canReuseCluster { err = func() error { l.PrintfCtx(ctx, "Using existing cluster: %s (arch=%q). Wiping", c.name, c.arch) - if err := c.WipeE(ctx, l); err != nil { + if err := c.WipeE(ctx, l, false /* preserveCerts */); err != nil { return err } if err := c.RunE(ctx, c.All(), "rm -rf "+perfArtifactsDir); err != nil { diff --git a/pkg/cmd/roachtest/tests/allocation_bench.go b/pkg/cmd/roachtest/tests/allocation_bench.go index 77e866672b40..ab657bdfa3ee 100644 --- a/pkg/cmd/roachtest/tests/allocation_bench.go +++ b/pkg/cmd/roachtest/tests/allocation_bench.go @@ -323,7 +323,7 @@ func setupStatCollector( if err := c.StopGrafana(ctx, t.L(), t.ArtifactsDir()); err != nil { t.L().ErrorfCtx(ctx, "Error(s) shutting down prom/grafana %s", err) } - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) } promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), cfg) diff --git a/pkg/cmd/roachtest/tests/autoupgrade.go b/pkg/cmd/roachtest/tests/autoupgrade.go index d1f61325533c..2f9eeaf65424 100644 --- a/pkg/cmd/roachtest/tests/autoupgrade.go +++ b/pkg/cmd/roachtest/tests/autoupgrade.go @@ -248,7 +248,7 @@ func registerAutoUpgrade(r registry.Registry) { // Wipe n3 to exclude it from the dead node check the roachtest harness // will perform after the test. - c.Wipe(ctx, c.Node(nodeDecommissioned)) + c.Wipe(ctx, false /* preserveCerts */, c.Node(nodeDecommissioned)) } r.Add(registry.TestSpec{ diff --git a/pkg/cmd/roachtest/tests/clock_jump_crash.go b/pkg/cmd/roachtest/tests/clock_jump_crash.go index 21d1a4747bbb..341edd3a1d09 100644 --- a/pkg/cmd/roachtest/tests/clock_jump_crash.go +++ b/pkg/cmd/roachtest/tests/clock_jump_crash.go @@ -39,7 +39,7 @@ func runClockJump(ctx context.Context, t test.Test, c cluster.Cluster, tc clockJ if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) } - c.Wipe(ctx) + 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 379ec398c548..469dda8032d8 100644 --- a/pkg/cmd/roachtest/tests/clock_monotonic.go +++ b/pkg/cmd/roachtest/tests/clock_monotonic.go @@ -41,7 +41,7 @@ func runClockMonotonicity( if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) } - c.Wipe(ctx) + 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 5f5165eff18b..45e5b0b6ebd1 100644 --- a/pkg/cmd/roachtest/tests/cluster_init.go +++ b/pkg/cmd/roachtest/tests/cluster_init.go @@ -52,7 +52,7 @@ func runClusterInit(ctx context.Context, t test.Test, c cluster.Cluster) { // default to just the first node) and we want to make sure that we're not // relying on it. for _, initNode := range []int{2, 1} { - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) t.L().Printf("starting test with init node %d", initNode) startOpts := option.DefaultStartOpts() diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index 58fc0eeee38b..5129f0fbcfa3 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -746,7 +746,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust } // Now stop+wipe them for good to keep the logs simple and the dead node detector happy. c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Nodes(targetNodeA, targetNodeB)) - c.Wipe(ctx, c.Nodes(targetNodeA, targetNodeB)) + c.Wipe(ctx, false /* preserveCerts */, c.Nodes(targetNodeA, targetNodeB)) } } @@ -871,7 +871,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust t.L().Printf("wiping n%d and adding it back to the cluster as a new node\n", targetNode) c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Node(targetNode)) - c.Wipe(ctx, c.Node(targetNode)) + c.Wipe(ctx, false /*preserveCerts */, c.Node(targetNode)) joinNode := h.getRandNode() internalAddrs, err := c.InternalAddr(ctx, t.L(), c.Node(joinNode)) diff --git a/pkg/cmd/roachtest/tests/decommission_self.go b/pkg/cmd/roachtest/tests/decommission_self.go index db39d031a3b6..fe753af542d9 100644 --- a/pkg/cmd/roachtest/tests/decommission_self.go +++ b/pkg/cmd/roachtest/tests/decommission_self.go @@ -31,7 +31,7 @@ func runDecommissionSelf(ctx context.Context, t test.Test, c cluster.Cluster) { // Stop n2 and exclude it from post-test consistency checks, // as this node can't contact cluster any more and operations // on it will hang. - u.c.Wipe(ctx, c.Node(2)) + u.c.Wipe(ctx, false /* preserveCerts */, c.Node(2)) }, checkOneMembership(1, "decommissioned"), ) diff --git a/pkg/cmd/roachtest/tests/inconsistency.go b/pkg/cmd/roachtest/tests/inconsistency.go index 9ac12a566e0d..2266889f1375 100644 --- a/pkg/cmd/roachtest/tests/inconsistency.go +++ b/pkg/cmd/roachtest/tests/inconsistency.go @@ -160,5 +160,5 @@ func runInconsistency(ctx context.Context, t test.Test, c cluster.Cluster) { // roachtest checks that no nodes are down when the test finishes, but in this // case we have a down node that we can't restart. Remove the data dir, which // tells roachtest to ignore this node. - c.Wipe(ctx, c.Node(1)) + c.Wipe(ctx, false /* preserveCerts */, c.Node(1)) } diff --git a/pkg/cmd/roachtest/tests/kv.go b/pkg/cmd/roachtest/tests/kv.go index 864ef2a600f2..38d79d17382f 100644 --- a/pkg/cmd/roachtest/tests/kv.go +++ b/pkg/cmd/roachtest/tests/kv.go @@ -767,7 +767,7 @@ func registerKVScalability(r registry.Registry) { const maxPerNodeConcurrency = 64 for i := nodes; i <= nodes*maxPerNodeConcurrency; i += nodes { i := i // capture loop variable - c.Wipe(ctx, c.Range(1, nodes)) + c.Wipe(ctx, false /* preserveCerts */, c.Range(1, nodes)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, nodes)) t.Status("running workload") diff --git a/pkg/cmd/roachtest/tests/kvbench.go b/pkg/cmd/roachtest/tests/kvbench.go index 108d85d45901..c90c52a51bcc 100644 --- a/pkg/cmd/roachtest/tests/kvbench.go +++ b/pkg/cmd/roachtest/tests/kvbench.go @@ -229,7 +229,7 @@ func runKVBench(ctx context.Context, t test.Test, c cluster.Cluster, b kvBenchSp // Wipe cluster before starting a new run because factors like load-based // splitting can significantly change the underlying layout of the table and // affect benchmark results. - c.Wipe(ctx, roachNodes) + c.Wipe(ctx, false /* preserveCerts */, roachNodes) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes) time.Sleep(restartWait) diff --git a/pkg/cmd/roachtest/tests/mixed_version_backup.go b/pkg/cmd/roachtest/tests/mixed_version_backup.go index 7ca5f4a9ca84..cd927aa121c5 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_backup.go +++ b/pkg/cmd/roachtest/tests/mixed_version_backup.go @@ -1964,7 +1964,7 @@ func (mvb *mixedVersionBackup) resetCluster( ctx context.Context, l *logger.Logger, version string, ) error { l.Printf("resetting cluster using version %s", version) - if err := mvb.cluster.WipeE(ctx, l, mvb.roachNodes); err != nil { + if err := mvb.cluster.WipeE(ctx, l, true /* preserveCerts */, mvb.roachNodes); err != nil { return fmt.Errorf("failed to wipe cluster: %w", err) } diff --git a/pkg/cmd/roachtest/tests/multitenant_tpch.go b/pkg/cmd/roachtest/tests/multitenant_tpch.go index ad0e44c78d67..e444e458752b 100644 --- a/pkg/cmd/roachtest/tests/multitenant_tpch.go +++ b/pkg/cmd/roachtest/tests/multitenant_tpch.go @@ -82,7 +82,7 @@ func runMultiTenantTPCH( // Restart and wipe the cluster to remove advantage of the second TPCH run. c.Stop(ctx, t.L(), option.DefaultStopOpts()) - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) start() singleTenantConn = c.Conn(ctx, t.L(), 1) // Disable merge queue in the system tenant. diff --git a/pkg/cmd/roachtest/tests/query_comparison_util.go b/pkg/cmd/roachtest/tests/query_comparison_util.go index 44be42337840..85cdad3ff35d 100644 --- a/pkg/cmd/roachtest/tests/query_comparison_util.go +++ b/pkg/cmd/roachtest/tests/query_comparison_util.go @@ -82,7 +82,7 @@ func runQueryComparison( return } c.Stop(clusterCtx, t.L(), option.DefaultStopOpts()) - c.Wipe(clusterCtx) + c.Wipe(clusterCtx, false /* preserveCerts */) } } diff --git a/pkg/cmd/roachtest/tests/rapid_restart.go b/pkg/cmd/roachtest/tests/rapid_restart.go index 1533fea72459..52d513c2bade 100644 --- a/pkg/cmd/roachtest/tests/rapid_restart.go +++ b/pkg/cmd/roachtest/tests/rapid_restart.go @@ -39,7 +39,7 @@ func runRapidRestart(ctx context.Context, t test.Test, c cluster.Cluster) { return timeutil.Now().After(deadline) } for j := 1; !done(); j++ { - c.Wipe(ctx, node) + c.Wipe(ctx, false /* preserveCerts */, node) // The first 2 iterations we start the cockroach node and kill it right // away. The 3rd iteration we let cockroach run so that we can check after @@ -97,5 +97,5 @@ func runRapidRestart(ctx context.Context, t test.Test, c cluster.Cluster) { // that consistency checks can be run, but in this case there's not much // there in the first place anyway. c.Stop(ctx, t.L(), option.DefaultStopOpts(), node) - c.Wipe(ctx, node) + c.Wipe(ctx, false /* preserveCerts */, node) } diff --git a/pkg/cmd/roachtest/tests/sstable_corruption.go b/pkg/cmd/roachtest/tests/sstable_corruption.go index 79d47b719982..42924c5cd1a3 100644 --- a/pkg/cmd/roachtest/tests/sstable_corruption.go +++ b/pkg/cmd/roachtest/tests/sstable_corruption.go @@ -127,7 +127,7 @@ func runSSTableCorruption(ctx context.Context, t test.Test, c cluster.Cluster) { if err := c.StartE(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), crdbNodes); err != nil { // Node detected corruption on start and crashed. This is good. No need // to run workload; the test is complete. - _ = c.WipeE(ctx, t.L(), corruptNodes) + _ = c.WipeE(ctx, t.L(), false /* preserveCerts */, corruptNodes) return } @@ -174,7 +174,7 @@ func runSSTableCorruption(ctx context.Context, t test.Test, c cluster.Cluster) { } // Exempt corrupted nodes from roachtest harness' post-test liveness checks. - _ = c.WipeE(ctx, t.L(), corruptNodes) + _ = c.WipeE(ctx, t.L(), false /* preserveCerts */, corruptNodes) } func registerSSTableCorruption(r registry.Registry) { diff --git a/pkg/cmd/roachtest/tests/tlp.go b/pkg/cmd/roachtest/tests/tlp.go index 713829f91360..2ee359cd1d23 100644 --- a/pkg/cmd/roachtest/tests/tlp.go +++ b/pkg/cmd/roachtest/tests/tlp.go @@ -82,7 +82,7 @@ func runTLP(ctx context.Context, t test.Test, c cluster.Cluster) { return } c.Stop(ctx, t.L(), option.DefaultStopOpts()) - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) } } diff --git a/pkg/cmd/roachtest/tests/tpc_utils.go b/pkg/cmd/roachtest/tests/tpc_utils.go index 640bd2885a4e..9d12197ba5dc 100644 --- a/pkg/cmd/roachtest/tests/tpc_utils.go +++ b/pkg/cmd/roachtest/tests/tpc_utils.go @@ -88,7 +88,7 @@ func loadTPCHDataset( // If the scale factor was smaller than the required scale factor, wipe the // cluster and restore. m.ExpectDeaths(int32(c.Spec().NodeCount)) - c.Wipe(ctx, roachNodes) + c.Wipe(ctx, false /* preserveCerts */, roachNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes) m.ResetDeaths() } else if pqErr := (*pq.Error)(nil); !(errors.As(err, &pqErr) && diff --git a/pkg/cmd/roachtest/tests/tpcc.go b/pkg/cmd/roachtest/tests/tpcc.go index 1edcc85aa0b7..a461290bbaaf 100644 --- a/pkg/cmd/roachtest/tests/tpcc.go +++ b/pkg/cmd/roachtest/tests/tpcc.go @@ -1169,7 +1169,7 @@ func loadTPCCBench( // If the dataset exists but is not large enough, wipe the cluster // before restoring. - c.Wipe(ctx, roachNodes) + c.Wipe(ctx, false /* preserveCerts */, roachNodes) startOpts, settings := b.startOpts() c.Start(ctx, t.L(), startOpts, settings, roachNodes) } else if pqErr := (*pq.Error)(nil); !(errors.As(err, &pqErr) && diff --git a/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go b/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go index 90ed4d0941db..e8b613657c39 100644 --- a/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go +++ b/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go @@ -75,7 +75,7 @@ func registerValidateSystemSchemaAfterVersionUpgrade(r registry.Registry) { // Wipe nodes in this test's cluster. wipeClusterStep := func(nodes option.NodeListOption) versionStep { return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - u.c.Wipe(ctx, nodes) + u.c.Wipe(ctx, false /* preserveCerts */, nodes) } } diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index 2e2f458442d5..92faae78db46 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/mixedversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -247,7 +248,9 @@ func uploadAndStartFromCheckpointFixture(nodes option.NodeListOption, v string) startOpts := option.DefaultStartOpts() // NB: can't start sequentially since cluster already bootstrapped. startOpts.RoachprodOpts.Sequential = false - if err := clusterupgrade.StartWithBinary(ctx, t.L(), u.c, nodes, binary, startOpts); err != nil { + if err := clusterupgrade.StartWithSettings( + ctx, t.L(), u.c, nodes, startOpts, install.BinaryOption(binary), + ); err != nil { t.Fatal(err) } } @@ -259,7 +262,9 @@ func uploadAndStart(nodes option.NodeListOption, v string) versionStep { startOpts := option.DefaultStartOpts() // NB: can't start sequentially since cluster already bootstrapped. startOpts.RoachprodOpts.Sequential = false - if err := clusterupgrade.StartWithBinary(ctx, t.L(), u.c, nodes, binary, startOpts); err != nil { + if err := clusterupgrade.StartWithSettings( + ctx, t.L(), u.c, nodes, startOpts, install.BinaryOption(binary), + ); err != nil { t.Fatal(err) } }