Skip to content

Commit

Permalink
Merge #74174
Browse files Browse the repository at this point in the history
74174: logictest: randomly backup/restore in logic tests r=stevendanna a=adityamaru

This commit introduces a new configuration to logic tests that will
randomly backup the cluster, re-create the cluster and restore that
backup to the new cluster.

This is primarily beneficial since it runs backup/restore under a much
more comprehensive set of SQL states.

This configuration is currently disabled via the
`COCKROACH_LOGIC_TEST_BACKUP_RESTORE_PROBABILITY` environment variable
being set to 0. The goal is to set up a nightly that
will set this var to a non-zero number for a growing set of logic
tests.

Co-authored-by: Paul Bardea <[email protected]>

Closes #54060.

Release note: None

Release justification: non-production code changes, adds a logictest
config that randomly runs backup and restore between existing logic tests.
This is currently disabled.

Co-authored-by: Paul Bardea <[email protected]>
  • Loading branch information
craig[bot] and pbardea committed Mar 1, 2022
2 parents 8eb279a + 05e9034 commit 12c968f
Show file tree
Hide file tree
Showing 2 changed files with 209 additions and 4 deletions.
1 change: 1 addition & 0 deletions pkg/sql/logictest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ go_library(
"//pkg/util/tracing",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_kr_pretty//:pretty",
"@com_github_lib_pq//:pq",
"@com_github_pmezard_go_difflib//difflib",
"@com_github_stretchr_testify//require",
Expand Down
212 changes: 208 additions & 4 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
"github.com/kr/pretty"
"github.com/lib/pq"
"github.com/pmezard/go-difflib/difflib"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -511,6 +512,9 @@ type testClusterConfig struct {
// localities is set if nodes should be set to a particular locality.
// Nodes are 1-indexed.
localities map[int]roachpb.Locality
// backupRestoreProbability will periodically backup the cluster and restore
// it's state to a new cluster at random points during a logic test.
backupRestoreProbability float64
}

const threeNodeTenantConfigName = "3node-tenant"
Expand Down Expand Up @@ -770,6 +774,18 @@ var logicTestConfigs = []testClusterConfig{
binaryVersion: roachpb.Version{Major: 22, Minor: 1},
disableUpgrade: true,
},
{
// 3node-backup is a config that periodically performs a cluster backup,
// and restores that backup into a new cluster before continuing the test.
// This config can only be run with a CCL binary, so is a noop if run
// through the normal logictest command.
// To run a logic test with this config as a directive, run:
// make test PKG=./pkg/ccl/logictestccl TESTS=TestBackupRestoreLogic//<test_name>
name: "3node-backup",
numNodes: 3,
backupRestoreProbability: envutil.EnvOrDefaultFloat64("COCKROACH_LOGIC_TEST_BACKUP_RESTORE_PROBABILITY", 0.0),
isCCLConfig: true,
},
}

// An index in the above slice.
Expand Down Expand Up @@ -1194,6 +1210,16 @@ type logicTest struct {
subtestT *testing.T
rng *rand.Rand
cfg testClusterConfig
// serverArgs are the parameters used to create a cluster for this test.
// They are persisted since a cluster can be recreated throughout the
// lifetime of the test and we should create all clusters with the same
// arguments.
serverArgs *TestServerArgs
// clusterOpts are the options used to create a cluster for this test.
// They are persisted since a cluster can be recreated throughout the
// lifetime of the test and we should create all clusters with the same
// arguments.
clusterOpts []clusterOpt
// cluster is the test cluster against which we are testing. This cluster
// may be reset during the lifetime of the test.
cluster serverutils.TestClusterInterface
Expand Down Expand Up @@ -1723,19 +1749,33 @@ func (t *logicTest) shutdownCluster() {
t.db = nil
}

// resetCluster cleans up the current cluster, and creates a fresh one.
func (t *logicTest) resetCluster() {
t.shutdownCluster()
if t.serverArgs == nil {
// We expect the server args to be persisted to the test during test
// setup.
t.Fatal("resetting the cluster before server args were set")
}
serverArgs := *t.serverArgs
t.newCluster(serverArgs, t.clusterOpts)
}

// setup creates the initial cluster for the logic test and populates the
// relevant fields on logicTest. It is expected to be called only once (per test
// file), and before processing any test files - unless a mock logicTest is
// created (see parallelTest.processTestFile).
func (t *logicTest) setup(cfg testClusterConfig, serverArgs TestServerArgs, opts []clusterOpt) {
t.cfg = cfg
t.serverArgs = &serverArgs
t.clusterOpts = opts[:]
// TODO(pmattis): Add a flag to make it easy to run the tests against a local
// MySQL or Postgres instance.
tempExternalIODir, tempExternalIODirCleanup := testutils.TempDir(t.rootT)
t.sharedIODir = tempExternalIODir
t.testCleanupFuncs = append(t.testCleanupFuncs, tempExternalIODirCleanup)

t.newCluster(serverArgs, opts)
t.newCluster(serverArgs, t.clusterOpts)

// Only create the test database on the initial cluster, since cluster restore
// expects an empty cluster.
Expand Down Expand Up @@ -1991,6 +2031,9 @@ type subtestDetails struct {
}

func (t *logicTest) processTestFile(path string, config testClusterConfig) error {
rng, seed := randutil.NewPseudoRand()
t.outf("rng seed: %d\n", seed)

subtests, err := fetchSubtests(path)
if err != nil {
return err
Expand All @@ -2008,7 +2051,7 @@ func (t *logicTest) processTestFile(path string, config testClusterConfig) error
// If subtest has no name, then it is not a subtest, so just run the lines
// in the overall test. Note that this can only happen in the first subtest.
if len(subtest.name) == 0 {
if err := t.processSubtest(subtest, path); err != nil {
if err := t.processSubtest(subtest, path, config, rng); err != nil {
return err
}
} else {
Expand All @@ -2018,7 +2061,7 @@ func (t *logicTest) processTestFile(path string, config testClusterConfig) error
defer func() {
t.subtestT = nil
}()
if err := t.processSubtest(subtest, path); err != nil {
if err := t.processSubtest(subtest, path, config, rng); err != nil {
t.Error(err)
}
})
Expand All @@ -2044,6 +2087,162 @@ func (t *logicTest) processTestFile(path string, config testClusterConfig) error
return nil
}

func (t *logicTest) hasOpenTxns(ctx context.Context) bool {
for _, user := range t.clients {
existingTxnPriority := "NORMAL"
err := user.QueryRow("SHOW TRANSACTION PRIORITY").Scan(&existingTxnPriority)
if err != nil {
// Ignore an error if we are unable to see transaction priority.
log.Warningf(ctx, "failed to check txn priority with %v", err)
continue
}
if _, err := user.Exec("SET TRANSACTION PRIORITY NORMAL;"); !testutils.IsError(err, "there is no transaction in progress") {
// Reset the txn priority to what it was before we checked for open txns.
_, err := user.Exec(fmt.Sprintf(`SET TRANSACTION PRIORITY %s`, existingTxnPriority))
if err != nil {
log.Warningf(ctx, "failed to reset txn priority with %v", err)
}
return true
}
}
return false
}

// maybeBackupRestore will randomly issue a cluster backup, create a new
// cluster, and restore that backup to the cluster before continuing the test.
// The probability of executing a backup and restore is specified in the
// testClusterConfig.
func (t *logicTest) maybeBackupRestore(rng *rand.Rand, config testClusterConfig) error {
if config.backupRestoreProbability != 0 && !config.isCCLConfig {
return errors.Newf("logic test config %s specifies a backup restore probability but is not CCL",
config.name)
}

// We decide if we want to take a backup here based on a probability
// specified in the logic test config.
if rng.Float64() > config.backupRestoreProbability {
return nil
}

// Check if any users have open transactions in the logictest. If they do, we
// do not want to teardown the cluster and create a new one as it might
// interfere with what the logictest is trying to test.
//
// We could perhaps make this smarter and perform the backup after the
// transaction is close.
if t.hasOpenTxns(context.Background()) {
return nil
}

oldUser := t.user
defer func() {
t.setUser(oldUser, 0 /* nodeIdxOverride */)
}()

// To restore the same state in for the logic test, we need to restore the
// data and the session state. The session state includes things like session
// variables that are set for every session that is open.
//
// TODO(adityamaru): A better approach might be to wipe the cluster once we
// have a command that enables this. That way all of the session data will not
// be lost in the process of creating a new cluster.
users := make([]string, 0, len(t.clients))
userToHexSession := make(map[string]string, len(t.clients))
userToSessionVars := make(map[string]map[string]string, len(t.clients))
for user := range t.clients {
t.setUser(user, 0 /* nodeIdxOverride */)
users = append(users, user)

// Serialize session variables.
var userSession string
var err error
if err = t.db.QueryRow(`SELECT encode(crdb_internal.serialize_session(), 'hex')`).Scan(&userSession); err == nil {
userToHexSession[user] = userSession
continue
}
log.Warningf(context.Background(), "failed to serialize session: %+v", err)

// If we failed to serialize the session variables, lets save the output of
// `SHOW ALL`. This usually happens if the session contains prepared
// statements or portals that cause the `serialize_session()` to fail.
//
// Saving the session variables in this manner does not guarantee the test
// will succeed since there are no ordering semantics when we go to apply
// them. There are some session variables that need to be applied before
// others for them to be valid. Thus, it is strictly better to use
// `serialize/deserialize_session()`, this "hack" just gives the test one
// more chance to succeed.
userSessionVars := make(map[string]string)
existingSessionVars, err := t.db.Query("SHOW ALL")
if err != nil {
return err
}
for existingSessionVars.Next() {
var key, value string
if err := existingSessionVars.Scan(&key, &value); err != nil {
return errors.Wrap(err, "scanning session variables")
}
userSessionVars[key] = value
}
userToSessionVars[user] = userSessionVars
}

backupLocation := fmt.Sprintf("nodelocal://1/logic-test-backup-%s",
strconv.FormatInt(timeutil.Now().UnixNano(), 10))

// Perform the backup and restore as root.
t.setUser(security.RootUser, 0 /* nodeIdxOverride */)

if _, err := t.db.Exec(fmt.Sprintf("BACKUP TO '%s'", backupLocation)); err != nil {
return errors.Wrap(err, "backing up cluster")
}

// Create a new cluster. Perhaps this can be updated to just wipe the exiting
// cluster once we have the ability to easily wipe a cluster through SQL.
t.resetCluster()

// Run the restore as root.
t.setUser(security.RootUser, 0 /* nodeIdxOverride */)
if _, err := t.db.Exec(fmt.Sprintf("RESTORE FROM '%s'", backupLocation)); err != nil {
return errors.Wrap(err, "restoring cluster")
}

// Restore the session state that was in the old cluster.

// Create new connections for the existing users, and restore the session
// variables that we collected.
for _, user := range users {
// Call setUser for every user to create the connection for that user.
t.setUser(user, 0 /* nodeIdxOverride */)

if userSession, ok := userToHexSession[user]; ok {
if _, err := t.db.Exec(fmt.Sprintf(`SELECT crdb_internal.deserialize_session(decode('%s', 'hex'))`, userSession)); err != nil {
return errors.Wrapf(err, "deserializing session")
}
} else if vars, ok := userToSessionVars[user]; ok {
// We now attempt to restore the session variables that were set on the
// backing up cluster. These are not included in the backup restore and so
// have to be restored manually.
for key, value := range vars {
// First try setting the cluster setting as a string.
if _, err := t.db.Exec(fmt.Sprintf("SET %s='%s'", key, value)); err != nil {
// If it fails, try setting the value as an int.
log.Infof(context.Background(), "setting session variable as string failed (err: %v), trying as int", pretty.Formatter(err))
if _, err := t.db.Exec(fmt.Sprintf("SET %s=%s", key, value)); err != nil {
// Some cluster settings can't be set at all, so ignore these errors.
// If a setting that we needed could not be restored, we expect the
// logic test to fail and let us know.
log.Infof(context.Background(), "setting session variable as int failed: %v (continuing anyway)", pretty.Formatter(err))
continue
}
}
}
}
}

return nil
}

// fetchSubtests reads through the test file and splices it into subtest chunks.
// If there is no subtest, the output will only contain a single entry.
func fetchSubtests(path string) ([]subtestDetails, error) {
Expand Down Expand Up @@ -2092,7 +2291,9 @@ func fetchSubtests(path string) ([]subtestDetails, error) {
return subtests, nil
}

func (t *logicTest) processSubtest(subtest subtestDetails, path string) error {
func (t *logicTest) processSubtest(
subtest subtestDetails, path string, config testClusterConfig, rng *rand.Rand,
) error {
defer t.traceStop()

s := newLineScanner(subtest.buffer)
Expand Down Expand Up @@ -2122,6 +2323,9 @@ func (t *logicTest) processSubtest(subtest subtestDetails, path string) error {
path, s.line+subtest.lineLineIndexIntoFile,
)
}
if err := t.maybeBackupRestore(rng, config); err != nil {
return err
}
switch cmd {
case "repeat":
// A line "repeat X" makes the test repeat the following statement or query X times.
Expand Down

0 comments on commit 12c968f

Please sign in to comment.