Skip to content

Commit

Permalink
roachtest: update mixedversion to always use secure clusters
Browse files Browse the repository at this point in the history
Secure clusters are closer to production deployments and also allow
us to tests features that we couldn't before, like creating new users
with passwords during a test, and then performing SQL operations with
those users.

Epic: CRDB-19321

Release note: None
  • Loading branch information
renatolabs committed Jun 28, 2023
1 parent 2c81156 commit cf171a6
Show file tree
Hide file tree
Showing 7 changed files with 57 additions and 29 deletions.
11 changes: 7 additions & 4 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -1499,10 +1499,13 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger, dest
// Ignore the files in the the log directory; we pull the logs separately anyway
// so this would only cause duplication.
excludeFiles := "*.log,*.txt,*.pprof"
cmd := fmt.Sprintf(
"%s debug zip --include-range-info --exclude-files='%s' --url {pgurl:%d} %s",
defaultCockroachPath, excludeFiles, i, zipName,
)
cmd := roachtestutil.NewCommand("%s debug zip", defaultCockroachPath).
Option("include-range-info").
Flag("exclude-files", fmt.Sprintf("'%s'", excludeFiles)).
Flag("url", fmt.Sprintf("{pgurl:%d}", i)).
MaybeFlag(c.IsSecure(), "certs-dir", "certs").
Arg(zipName).
String()
if err := c.RunE(ctx, c.Node(i), cmd); err != nil {
l.Printf("%s debug zip failed on node %d: %v", defaultCockroachPath, i, err)
if i < c.spec.NodeCount {
Expand Down
15 changes: 9 additions & 6 deletions pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,17 +137,17 @@ func InstallFixtures(
return nil
}

// StartWithBinary starts a cockroach binary, assumed to already be
// present in the nodes in the path given.
func StartWithBinary(
// StartWithSettings starts cockroach and constructs settings according
// to the setting options passed.
func StartWithSettings(
ctx context.Context,
l *logger.Logger,
c cluster.Cluster,
nodes option.NodeListOption,
binaryPath string,
startOpts option.StartOpts,
opts ...install.ClusterSettingOption,
) error {
settings := install.MakeClusterSettings(install.BinaryOption(binaryPath))
settings := install.MakeClusterSettings(opts...)
return c.StartE(ctx, l, startOpts, settings, nodes)
}

Expand All @@ -171,6 +171,7 @@ func RestartNodesWithNewBinary(
nodes option.NodeListOption,
startOpts option.StartOpts,
newVersion string,
settings ...install.ClusterSettingOption,
) error {
// NB: We could technically stage the binary on all nodes before
// restarting each one, but on Unix it's invalid to write to an
Expand Down Expand Up @@ -200,7 +201,9 @@ func RestartNodesWithNewBinary(
if err != nil {
return err
}
if err := StartWithBinary(ctx, l, c, c.Node(node), binary, startOpts); err != nil {
if err := StartWithSettings(
ctx, l, c, c.Node(node), startOpts, append(settings, install.BinaryOption(binary))...,
); err != nil {
return err
}

Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
"//pkg/cmd/roachtest/roachtestutil/clusterupgrade",
"//pkg/cmd/roachtest/test",
"//pkg/roachpb",
"//pkg/roachprod/install",
"//pkg/roachprod/logger",
"//pkg/util/ctxgroup",
"//pkg/util/randutil",
Expand Down
15 changes: 14 additions & 1 deletion pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/version"
Expand Down Expand Up @@ -114,6 +115,13 @@ var (
possibleDelaysMs = []int{
0, 50, 100, 200, 500,
}

// defaultClusterSettings is the set of cluster settings always
// passed to `clusterupgrade.StartWithSettings` when (re)starting
// nodes in a cluster.
defaultClusterSettings = []install.ClusterSettingOption{
install.SecureOption(true),
}
)

type (
Expand Down Expand Up @@ -483,7 +491,11 @@ func (s startFromCheckpointStep) Run(

startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.Sequential = false
return clusterupgrade.StartWithBinary(ctx, l, c, s.crdbNodes, binaryPath, startOpts)
clusterSettings := append(
append([]install.ClusterSettingOption{}, defaultClusterSettings...),
install.BinaryOption(binaryPath),
)
return clusterupgrade.StartWithSettings(ctx, l, c, s.crdbNodes, startOpts, clusterSettings...)
}

// uploadCurrentVersionStep uploads the current cockroach binary to
Expand Down Expand Up @@ -606,6 +618,7 @@ func (s restartWithNewBinaryStep) Run(
// scheduled backup if necessary.
option.DefaultStartOptsNoBackups(),
s.version,
defaultClusterSettings...,
)
}

Expand Down
7 changes: 4 additions & 3 deletions pkg/cmd/roachtest/tests/backup_fixtures.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -135,10 +136,10 @@ func (bd *backupDriver) prepareCluster(ctx context.Context) {
bd.sp.hardware.getCRDBNodes(), version)
require.NoError(bd.t, err)

require.NoError(bd.t, clusterupgrade.StartWithBinary(ctx, bd.t.L(), bd.c,
require.NoError(bd.t, clusterupgrade.StartWithSettings(ctx, bd.t.L(), bd.c,
bd.sp.hardware.getCRDBNodes(),
binaryPath,
option.DefaultStartOptsNoBackups()))
option.DefaultStartOptsNoBackups(),
install.BinaryOption(binaryPath)))

bd.assertCorrectCockroachBinary(ctx)
if !bd.sp.backup.ignoreExistingBackups {
Expand Down
10 changes: 6 additions & 4 deletions pkg/cmd/roachtest/tests/mixed_version_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
Expand Down Expand Up @@ -811,7 +812,7 @@ func (sc *systemTableContents) loadShowResults(

query := fmt.Sprintf("SELECT * FROM [%s]%s", showStmt, aostFor(timestamp))
showCmd := roachtestutil.NewCommand("%s sql", mixedversion.CurrentCockroachPath).
Option("insecure").
Flag("certs-dir", "certs").
Flag("e", fmt.Sprintf("%q", query)).
String()

Expand Down Expand Up @@ -1963,14 +1964,15 @@ func (mvb *mixedVersionBackup) verifyBackupCollection(
func (mvb *mixedVersionBackup) resetCluster(
ctx context.Context, l *logger.Logger, version string,
) error {
l.Printf("resetting cluster using version %s", version)
l.Printf("resetting cluster using version %q", clusterupgrade.VersionMsg(version))
if err := mvb.cluster.WipeE(ctx, l, true /* preserveCerts */, mvb.roachNodes); err != nil {
return fmt.Errorf("failed to wipe cluster: %w", err)
}

cockroachPath := clusterupgrade.BinaryPathFromVersion(version)
return clusterupgrade.StartWithBinary(
ctx, l, mvb.cluster, mvb.roachNodes, cockroachPath, option.DefaultStartOptsNoBackups(),
return clusterupgrade.StartWithSettings(
ctx, l, mvb.cluster, mvb.roachNodes, option.DefaultStartOptsNoBackups(),
install.BinaryOption(cockroachPath), install.SecureOption(true),
)
}

Expand Down
27 changes: 16 additions & 11 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,17 +101,22 @@ DROP TABLE splitmerge.t;
func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) {
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.All())
mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All())
mvt.OnStartup("setup schema changer workload", func(ctx context.Context, l *logger.Logger, r *rand.Rand, helper *mixedversion.Helper) error {
// Execute the workload init.
return c.RunE(ctx, c.All(), "./workload init schemachange")
})
mvt.InMixedVersion("run backup", func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error {
// Verify that backups can be created in various configurations. This is
// important to test because changes in system tables might cause backups to
// fail in mixed-version clusters.
dest := fmt.Sprintf("nodelocal://1/%d", timeutil.Now().UnixNano())
return h.Exec(rng, `BACKUP TO $1`, dest)
})
mvt.OnStartup(
"setup schema changer workload",
func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error {
node := h.RandomNode(rng, c.All())
l.Printf("executing workload init on node %d", node)
return c.RunE(ctx, c.Node(node), fmt.Sprintf("./workload init schemachange {pgurl%s}", c.All()))
})
mvt.InMixedVersion(
"run backup",
func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error {
// Verify that backups can be created in various configurations. This is
// important to test because changes in system tables might cause backups to
// fail in mixed-version clusters.
dest := fmt.Sprintf("nodelocal://1/%d", timeutil.Now().UnixNano())
return h.Exec(rng, `BACKUP TO $1`, dest)
})
mvt.InMixedVersion(
"test features",
func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error {
Expand Down

0 comments on commit cf171a6

Please sign in to comment.