Skip to content

Commit

Permalink
Merge #131467
Browse files Browse the repository at this point in the history
131467: roachtest: update `generate-fixtures` to use `clusterupgrade` functions r=srosenberg a=renatolabs

This commit updates the `generate-fixtures` roachtest, used to automate the generation of fixtures every release, so that it uses plain `clusterupgrade` functions instead of the `newUpgradeTest` API.

The old "step" functions only provided thin wrappers around the functionality of `clusterupgrade`. With this change, we are now able to completely remove the `newUpgradeTest` API which has been deprecated for a while in favor of the `mixedversion` framework.

Fixes: #110528

Release note: None

Co-authored-by: Renato Costa <[email protected]>
  • Loading branch information
craig[bot] and renatolabs committed Sep 27, 2024
2 parents 008333e + d08e2f1 commit 67dc7a1
Showing 1 changed file with 89 additions and 160 deletions.
249 changes: 89 additions & 160 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,13 @@ import (
gosql "database/sql"
"fmt"
"math/rand"
"runtime"
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
"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"
Expand Down Expand Up @@ -153,54 +151,6 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) {
mvt.Run()
}

func (u *versionUpgradeTest) run(ctx context.Context, t test.Test) {
defer func() {
for _, db := range u.conns {
_ = db.Close()
}
}()

for i, step := range u.steps {
if step != nil {
t.Status(fmt.Sprintf("versionUpgradeTest: starting step %d", i+1))
step(ctx, t, u)
}
}
}

type versionUpgradeTest struct {
goOS string
c cluster.Cluster
steps []versionStep

// Cache conns because opening one takes hundreds of ms, and we do it quite
// a lot.
conns []*gosql.DB
}

func newVersionUpgradeTest(c cluster.Cluster, steps ...versionStep) *versionUpgradeTest {
return &versionUpgradeTest{
goOS: ifLocal(c, runtime.GOOS, "linux"),
c: c,
steps: steps,
}
}

// Return a cached conn to the given node. Don't call .Close(), the test harness
// will do it.
func (u *versionUpgradeTest) conn(ctx context.Context, t test.Test, i int) *gosql.DB {
if u.conns == nil {
for _, i := range u.c.All() {
u.conns = append(u.conns, u.c.Conn(ctx, t.L(), i))
}
}
db := u.conns[i-1]
// Run a trivial query to shake out errors that can occur when the server has
// restarted in the meantime.
_ = db.PingContext(ctx)
return db
}

// uploadCockroach is a thin wrapper around
// `clusterupgrade.UploadCockroach` that calls t.Fatal if that call
// returns an error.
Expand All @@ -219,69 +169,6 @@ func uploadCockroach(
return path
}

func (u *versionUpgradeTest) binaryVersion(
ctx context.Context, t test.Test, i int,
) roachpb.Version {
db := u.conn(ctx, t, i)
v, err := clusterupgrade.BinaryVersion(ctx, db)
if err != nil {
t.Fatal(err)
}

return v
}

// versionStep is an isolated version migration on a running cluster.
type versionStep func(ctx context.Context, t test.Test, u *versionUpgradeTest)

func uploadAndStartFromCheckpointFixture(
nodes option.NodeListOption, v *clusterupgrade.Version,
) versionStep {
return func(ctx context.Context, t test.Test, u *versionUpgradeTest) {
if err := clusterupgrade.InstallFixtures(ctx, t.L(), u.c, nodes, v); err != nil {
t.Fatal(err)
}
binary := uploadCockroach(ctx, t, u.c, nodes, v)
startOpts := option.DefaultStartOpts()
if err := clusterupgrade.StartWithSettings(
ctx, t.L(), u.c, nodes, startOpts, install.BinaryOption(binary),
); err != nil {
t.Fatal(err)
}
}
}

// binaryUpgradeStep rolling-restarts the given nodes into the new binary
// version. Note that this does *not* wait for the cluster version to upgrade.
// Use a waitForUpgradeStep() for that.
func binaryUpgradeStep(
nodes option.NodeListOption, newVersion *clusterupgrade.Version,
) versionStep {
return func(ctx context.Context, t test.Test, u *versionUpgradeTest) {
if err := clusterupgrade.RestartNodesWithNewBinary(
ctx, t, t.L(), u.c, nodes, option.NewStartOpts(option.NoBackupSchedule), newVersion,
); err != nil {
t.Fatal(err)
}
}
}

// NB: this is intentionally kept separate from binaryUpgradeStep because we run
// feature tests between the steps, and we want to expose them (at least
// heuristically) to the real-world situation in which some nodes have already
// learned of a cluster version bump (from Gossip) where others haven't. This
// situation tends to exhibit unexpected behavior.
func waitForUpgradeStep(nodes option.NodeListOption) versionStep {
return func(ctx context.Context, t test.Test, u *versionUpgradeTest) {
dbFunc := func(node int) *gosql.DB { return u.conn(ctx, t, node) }
if err := clusterupgrade.WaitForClusterUpgrade(
ctx, t.L(), nodes, dbFunc, clusterupgrade.DefaultUpgradeTimeout,
); err != nil {
t.Fatal(err)
}
}
}

// makeVersionFixtureAndFatal creates fixtures from which we can test
// mixed-version clusters (i.e. version X mixing with X-1). The fixtures date
// back all the way to v1.0; when development begins on version X, we make a
Expand All @@ -293,6 +180,23 @@ func waitForUpgradeStep(nodes option.NodeListOption) versionStep {
func makeVersionFixtureAndFatal(
ctx context.Context, t test.Test, c cluster.Cluster, makeFixtureVersion string,
) {
// Manage connections to nodes and make sure to close any open
// connections at the end of the test.
conns := make(map[int]*gosql.DB)
dbFunc := func(node int) *gosql.DB {
if _, ok := conns[node]; !ok {
conns[node] = c.Conn(ctx, t.L(), node)
}

return conns[node]
}

defer func() {
for _, db := range conns {
db.Close()
}
}()

predecessorVersionStr, err := release.LatestPredecessor(version.MustParse(makeFixtureVersion))
if err != nil {
t.Fatal(err)
Expand All @@ -302,56 +206,82 @@ func makeVersionFixtureAndFatal(
t.L().Printf("making fixture for %s (starting at %s)", makeFixtureVersion, predecessorVersion)
fixtureVersion := clusterupgrade.MustParseVersion(makeFixtureVersion)

newVersionUpgradeTest(c,
// Start the cluster from a fixture. That fixture's cluster version may
// be at the predecessor version (though in practice it's fully up to
// date, if it was created via the checkpointer above), so add a
// waitForUpgradeStep to make sure we're upgraded all the way before
// moving on.
//
// See the comment on createCheckpoints for details on fixtures.
uploadAndStartFromCheckpointFixture(c.All(), predecessorVersion),
waitForUpgradeStep(c.All()),
t.L().Printf("installing fixtures")
if err := clusterupgrade.InstallFixtures(ctx, t.L(), c, c.All(), predecessorVersion); err != nil {
t.Fatalf("installing fixtures: %v", err)
}

t.L().Printf("uploading cockroach version %s", predecessorVersion)
binary, err := clusterupgrade.UploadCockroach(ctx, t, t.L(), c, c.All(), predecessorVersion)
if err != nil {
t.Fatalf("uploading cockroach: %v", err)
}

t.L().Printf("starting cockroach process")
if err := clusterupgrade.StartWithSettings(
ctx, t.L(), c, c.All(), option.DefaultStartOpts(), install.BinaryOption(binary),
); err != nil {
t.Fatalf("starting cockroach: %v", err)
}

// NB: at this point, cluster and binary version equal predecessorVersion,
// and auto-upgrades are on.
t.L().Printf("waiting for stable cluster version")
if err := clusterupgrade.WaitForClusterUpgrade(
ctx, t.L(), c.All(), dbFunc, clusterupgrade.DefaultUpgradeTimeout,
); err != nil {
t.Fatalf("waiting for cluster to reach version %s: %v", predecessorVersion, err)
}

binaryUpgradeStep(c.All(), fixtureVersion),
waitForUpgradeStep(c.All()),
t.L().Printf("restarting cluster to version %s", fixtureVersion)
if err := clusterupgrade.RestartNodesWithNewBinary(
ctx, t, t.L(), c, c.All(), option.NewStartOpts(option.NoBackupSchedule), fixtureVersion,
); err != nil {
t.Fatalf("restarting cluster to binary version %s: %v", fixtureVersion, err)
}

func(ctx context.Context, t test.Test, u *versionUpgradeTest) {
// If we're taking checkpoints, momentarily stop the cluster (we
// need to do that to get the checkpoints to reflect a
// consistent cluster state). The binary at this point will be
// the new one, but the cluster version was not explicitly
// bumped, though auto-update may have taken place already.
// For example, if newVersion is 2.1, the cluster version in
// the store directories may be 2.0 on some stores and 2.1 on
// the others (though if any are on 2.1, then that's what's
// stored in system.settings).
// This means that when we restart from that version, we're
// going to want to use the binary mentioned in the checkpoint,
// or at least one compatible with the *predecessor* of the
// checkpoint version. For example, for checkpoint-2.1, the
// cluster version might be 2.0, so we can only use the 2.0 or
// 2.1 binary, but not the 19.1 binary (as 19.1 and 2.0 are not
// compatible).
name := clusterupgrade.CheckpointName(u.binaryVersion(ctx, t, 1).String())
u.c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.All())
t.L().Printf("waiting for upgrade to %s to finalize", fixtureVersion)
if err := clusterupgrade.WaitForClusterUpgrade(
ctx, t.L(), c.All(), dbFunc, clusterupgrade.DefaultUpgradeTimeout,
); err != nil {
t.Fatalf("waiting for upgrade to %s to finalize: %v", fixtureVersion, err)
}

// If we're taking checkpoints, momentarily stop the cluster (we
// need to do that to get the checkpoints to reflect a
// consistent cluster state). The binary at this point will be
// the new one, but the cluster version was not explicitly
// bumped, though auto-update may have taken place already.
// For example, if newVersion is 2.1, the cluster version in
// the store directories may be 2.0 on some stores and 2.1 on
// the others (though if any are on 2.1, then that's what's
// stored in system.settings).
// This means that when we restart from that version, we're
// going to want to use the binary mentioned in the checkpoint,
// or at least one compatible with the *predecessor* of the
// checkpoint version. For example, for checkpoint-2.1, the
// cluster version might be 2.0, so we can only use the 2.0 or
// 2.1 binary, but not the 19.1 binary (as 19.1 and 2.0 are not
// compatible).
binaryVersion, err := clusterupgrade.BinaryVersion(ctx, dbFunc(1))
if err != nil {
t.Fatalf("fetching binary version on n1: %v", err)
}

binaryPath := clusterupgrade.CockroachPathForVersion(t, fixtureVersion)
c.Run(ctx, option.WithNodes(c.All()), binaryPath, "debug", "pebble", "db", "checkpoint",
"{store-dir}", "{store-dir}/"+name)
// The `cluster-bootstrapped` marker can already be found within
// store-dir, but the rocksdb checkpoint step above does not pick it
// up as it isn't recognized by RocksDB. We copy the marker
// manually, it's necessary for roachprod created clusters. See
// #54761.
c.Run(ctx, option.WithNodes(c.Node(1)), "cp", "{store-dir}/cluster-bootstrapped", "{store-dir}/"+name)
// Similar to the above - newer versions require the min version file to open a store.
c.Run(ctx, option.WithNodes(c.All()), "cp", fmt.Sprintf("{store-dir}/%s", storage.MinVersionFilename), "{store-dir}/"+name)
c.Run(ctx, option.WithNodes(c.All()), "tar", "-C", "{store-dir}/"+name, "-czf", "{log-dir}/"+name+".tgz", ".")
t.Fatalf(`successfully created checkpoints; failing test on purpose.
name := clusterupgrade.CheckpointName(binaryVersion.String())
c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.All())

binaryPath := clusterupgrade.CockroachPathForVersion(t, fixtureVersion)
c.Run(ctx, option.WithNodes(c.All()), binaryPath, "debug", "pebble", "db", "checkpoint",
"{store-dir}", "{store-dir}/"+name)
// The `cluster-bootstrapped` marker can already be found within
// store-dir, but the rocksdb checkpoint step above does not pick it
// up as it isn't recognized by RocksDB. We copy the marker
// manually, it's necessary for roachprod created clusters. See
// #54761.
c.Run(ctx, option.WithNodes(c.Node(1)), "cp", "{store-dir}/cluster-bootstrapped", "{store-dir}/"+name)
// Similar to the above - newer versions require the min version file to open a store.
c.Run(ctx, option.WithNodes(c.All()), "cp", fmt.Sprintf("{store-dir}/%s", storage.MinVersionFilename), "{store-dir}/"+name)
c.Run(ctx, option.WithNodes(c.All()), "tar", "-C", "{store-dir}/"+name, "-czf", "{log-dir}/"+name+".tgz", ".")
t.Fatalf(`successfully created checkpoints; failing test on purpose.
Invoke the following to move the archives to the right place and commit the
result:
Expand All @@ -362,5 +292,4 @@ for i in 1 2 3 4; do
pkg/cmd/roachtest/fixtures/${i}/
done
`)
}).run(ctx, t)
}

0 comments on commit 67dc7a1

Please sign in to comment.