Skip to content

Commit

Permalink
roachtest: introduce preserveCerts parameter to Wipe
Browse files Browse the repository at this point in the history
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
  • Loading branch information
renatolabs committed Jun 28, 2023
1 parent 16bb892 commit 2c81156
Show file tree
Hide file tree
Showing 23 changed files with 39 additions and 32 deletions.
12 changes: 7 additions & 5 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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")
}
Expand All @@ -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)
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/cluster/cluster_interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/allocation_bench.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/autoupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/clock_jump_crash.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/clock_monotonic.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cluster_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()

Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}
}

Expand Down Expand Up @@ -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))
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/decommission_self.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"),
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/inconsistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/kvbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/mixed_version_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/multitenant_tpch.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/query_comparison_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func runQueryComparison(
return
}
c.Stop(clusterCtx, t.L(), option.DefaultStopOpts())
c.Wipe(clusterCtx)
c.Wipe(clusterCtx, false /* preserveCerts */)
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/rapid_restart.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
}
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/sstable_corruption.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down Expand Up @@ -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) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/tlp.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 */)
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/tpc_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) &&
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/tpcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) &&
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}

Expand Down
9 changes: 7 additions & 2 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
}
}
Expand All @@ -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)
}
}
Expand Down

0 comments on commit 2c81156

Please sign in to comment.