diff --git a/pkg/ccl/backupccl/datadriven_test.go b/pkg/ccl/backupccl/datadriven_test.go index 1836e6b316c0..89c83d9a2338 100644 --- a/pkg/ccl/backupccl/datadriven_test.go +++ b/pkg/ccl/backupccl/datadriven_test.go @@ -82,16 +82,16 @@ var clusterVersionKeys = map[string]clusterversion.Key{ } type sqlDBKey struct { - server string - user string + name string + user string } type datadrivenTestState struct { - // clusters maps a name to its cluster + // cluster maps the user defined cluster name to its cluster clusters map[string]serverutils.TestClusterInterface - // servers maps a name to the first server in the cluster - servers map[string]serverutils.TestServerInterface + // firstNode maps the cluster name to the first node in the cluster + firstNode map[string]serverutils.TestServerInterface dataDirs map[string]string sqlDBs map[sqlDBKey]*gosql.DB jobTags map[string]jobspb.JobID @@ -104,7 +104,7 @@ type datadrivenTestState struct { func newDatadrivenTestState() datadrivenTestState { return datadrivenTestState{ clusters: make(map[string]serverutils.TestClusterInterface), - servers: make(map[string]serverutils.TestServerInterface), + firstNode: make(map[string]serverutils.TestServerInterface), dataDirs: make(map[string]string), sqlDBs: make(map[sqlDBKey]*gosql.DB), jobTags: make(map[string]jobspb.JobID), @@ -114,14 +114,14 @@ func newDatadrivenTestState() datadrivenTestState { } func (d *datadrivenTestState) cleanup(ctx context.Context, t *testing.T) { - // While the testCluster cleanupFns would close the dbConn and servers, close + // While the testCluster cleanupFns would close the dbConn and clusters, close // them manually to ensure all queries finish on tests that share these // resources. for _, db := range d.sqlDBs { backuputils.CheckForInvalidDescriptors(t, db) db.Close() } - for _, s := range d.servers { + for _, s := range d.firstNode { s.Stopper().Stop(ctx) } for _, f := range d.cleanupFns { @@ -130,7 +130,7 @@ func (d *datadrivenTestState) cleanup(ctx context.Context, t *testing.T) { d.noticeBuffer = nil } -type serverCfg struct { +type clusterCfg struct { name string iodir string nodes int @@ -142,7 +142,7 @@ type serverCfg struct { disableTenant bool } -func (d *datadrivenTestState) addServer(t *testing.T, cfg serverCfg) error { +func (d *datadrivenTestState) addCluster(t *testing.T, cfg clusterCfg) error { var tc serverutils.TestClusterInterface var cleanup func() params := base.TestClusterArgs{} @@ -205,27 +205,27 @@ func (d *datadrivenTestState) addServer(t *testing.T, cfg serverCfg) error { cleanup() } d.clusters[cfg.name] = tc - d.servers[cfg.name] = tc.Server(0) + d.firstNode[cfg.name] = tc.Server(0) d.dataDirs[cfg.name] = cfg.iodir d.cleanupFns = append(d.cleanupFns, cleanupFn) return nil } -func (d *datadrivenTestState) getIODir(t *testing.T, server string) string { - dir, ok := d.dataDirs[server] +func (d *datadrivenTestState) getIODir(t *testing.T, name string) string { + dir, ok := d.dataDirs[name] if !ok { - t.Fatalf("server %s does not exist", server) + t.Fatalf("cluster %s does not exist", name) } return dir } -func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) *gosql.DB { - key := sqlDBKey{server, user} +func (d *datadrivenTestState) getSQLDB(t *testing.T, name string, user string) *gosql.DB { + key := sqlDBKey{name, user} if db, ok := d.sqlDBs[key]; ok { return db } - addr := d.servers[server].ServingSQLAddr() + addr := d.firstNode[name].ServingSQLAddr() pgURL, cleanup := sqlutils.PGUrl(t, addr, "TestBackupRestoreDataDriven", url.User(user)) d.cleanupFns = append(d.cleanupFns, cleanup) @@ -251,14 +251,14 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) // commands. The test files are in testdata/backup-restore. The following // syntax is provided: // -// - "new-server name= [args]" -// Create a new server with the input name. +// - "new-cluster name= [args]" +// Create a new cluster with the input name. // // Supported arguments: // // - share-io-dir: can be specified to share an IO directory with an existing -// server. This is useful when restoring from a backup taken in another -// server. +// cluster. This is useful when restoring from a backup taken in another +// cluster. // // - allow-implicit-access: can be specified to set // `EnableNonAdminImplicitAndArbitraryOutbound` to true @@ -275,7 +275,7 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) // - splits: specifies the number of ranges the bank table is split into. // // - before-version=: creates a mixed version cluster where all -// nodes running the test server binary think the clusterVersion is one +// nodes running the test cluster binary think the clusterVersion is one // version before the passed in key. See cockroach_versions.go // for possible values. // @@ -284,13 +284,13 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) // - disable-tenant : ensures the test is never run in a multitenant environment by // setting testserverargs.DisableDefaultTestTenant to true. // -// - "upgrade-server version=" -// Upgrade the cluster version of the active server to the passed in +// - "upgrade-cluster version=" +// Upgrade the cluster version of the active cluster to the passed in // clusterVersion key. See cockroach_versions.go for possible values. // -// - "exec-sql [server=] [user=] [args]" -// Executes the input SQL query on the target server. By default, server is -// the last created server. +// - "exec-sql [cluster=] [user=] [args]" +// Executes the input SQL query on the target cluster. By default, cluster is +// the last created cluster. // // Supported arguments: // @@ -303,7 +303,7 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) // - ignore-notice: does not print out the notice that is buffered during // query execution. // -// - "query-sql [server=] [user=] [regex=]" +// - "query-sql [cluster=] [user=] [regex=]" // Executes the input SQL query and print the results. // // Supported arguments: @@ -318,7 +318,7 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) // - "reset" // Clear all state associated with the test. // -// - "job" [server=] [user=] [args] +// - "job" [cluster=] [user=] [args] // Executes job specific operations. // // Supported arguments: @@ -385,7 +385,7 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) // - "corrupt-backup" uri= // Finds the latest backup in the provided collection uri an flips a bit in one SST in the backup // -// - "link-backup" server= src-path= dest-path= +// - "link-backup" cluster= src-path= dest-path= // Creates a symlink from the testdata path to the file IO path, so that we // can restore precreated backup. src-path and dest-path are comma seperated // paths that will be joined. @@ -404,14 +404,14 @@ func TestDataDriven(t *testing.T) { ctx := context.Background() datadriven.Walk(t, testutils.TestDataPath(t, "backup-restore"), func(t *testing.T, path string) { - var lastCreatedServer string + var lastCreatedCluster string ds := newDatadrivenTestState() defer ds.cleanup(ctx, t) datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { execWithTagAndPausePoint := func(jobType jobspb.Type) string { const user = "root" - sqlDB := ds.getSQLDB(t, lastCreatedServer, user) + sqlDB := ds.getSQLDB(t, lastCreatedCluster, user) // First, run the schema change. _, err := sqlDB.Exec(d.Input) @@ -467,7 +467,7 @@ func TestDataDriven(t *testing.T) { ds = newDatadrivenTestState() return "" - case "new-server": + case "new-cluster": var name, shareDirWith, iodir, localities, beforeVersion, testingKnobCfg string var splits int var disableTenant bool @@ -509,8 +509,8 @@ func TestDataDriven(t *testing.T) { disableTenant = true } - lastCreatedServer = name - cfg := serverCfg{ + lastCreatedCluster = name + cfg := clusterCfg{ name: name, iodir: iodir, nodes: nodes, @@ -521,20 +521,20 @@ func TestDataDriven(t *testing.T) { testingKnobCfg: testingKnobCfg, disableTenant: disableTenant, } - err := ds.addServer(t, cfg) + err := ds.addCluster(t, cfg) if err != nil { return err.Error() } return "" - case "switch-server": + case "switch-cluster": var name string d.ScanArgs(t, "name", &name) - lastCreatedServer = name + lastCreatedCluster = name return "" - case "upgrade-server": - server := lastCreatedServer + case "upgrade-cluster": + cluster := lastCreatedCluster user := "root" var version string @@ -546,23 +546,23 @@ func TestDataDriven(t *testing.T) { t.Fatalf("clusterVersion %s does not exist in data driven global map", version) } clusterVersion := clusterversion.ByKey(key) - _, err := ds.getSQLDB(t, server, user).Exec("SET CLUSTER SETTING version = $1", clusterVersion.String()) + _, err := ds.getSQLDB(t, cluster, user).Exec("SET CLUSTER SETTING version = $1", clusterVersion.String()) require.NoError(t, err) return "" case "exec-sql": - server := lastCreatedServer + cluster := lastCreatedCluster user := "root" - if d.HasArg("server") { - d.ScanArgs(t, "server", &server) + if d.HasArg("cluster") { + d.ScanArgs(t, "cluster", &cluster) } if d.HasArg("user") { d.ScanArgs(t, "user", &user) } ds.noticeBuffer = nil - checkForClusterSetting(t, d.Input, ds.clusters[server].NumServers()) - d.Input = strings.ReplaceAll(d.Input, "http://COCKROACH_TEST_HTTP_SERVER/", httpAddr) - _, err := ds.getSQLDB(t, server, user).Exec(d.Input) + checkForClusterSetting(t, d.Input, ds.clusters[cluster].NumServers()) + d.Input = strings.ReplaceAll(d.Input, "http://COCKROACH_TEST_HTTP_server/", httpAddr) + _, err := ds.getSQLDB(t, cluster, user).Exec(d.Input) ret := ds.noticeBuffer if d.HasArg("ignore-notice") { @@ -577,11 +577,11 @@ func TestDataDriven(t *testing.T) { // Find job ID of the pausepoint job. var jobID jobspb.JobID require.NoError(t, - ds.getSQLDB(t, server, user).QueryRow( + ds.getSQLDB(t, cluster, user).QueryRow( `SELECT job_id FROM [SHOW JOBS] ORDER BY created DESC LIMIT 1`).Scan(&jobID)) fmt.Printf("expecting pausepoint, found job ID %d\n\n\n", jobID) - runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, server, user)) + runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, cluster, user)) jobutils.WaitForJobToPause(t, runner, jobID) ret = append(ds.noticeBuffer, "job paused at pausepoint") ret = append(ret, "") @@ -627,16 +627,16 @@ func TestDataDriven(t *testing.T) { return strings.Join(ret, "\n") case "query-sql": - server := lastCreatedServer + cluster := lastCreatedCluster user := "root" - if d.HasArg("server") { - d.ScanArgs(t, "server", &server) + if d.HasArg("cluster") { + d.ScanArgs(t, "cluster", &cluster) } if d.HasArg("user") { d.ScanArgs(t, "user", &user) } - checkForClusterSetting(t, d.Input, ds.clusters[server].NumServers()) - rows, err := ds.getSQLDB(t, server, user).Query(d.Input) + checkForClusterSetting(t, d.Input, ds.clusters[cluster].NumServers()) + rows, err := ds.getSQLDB(t, cluster, user).Query(d.Input) if err != nil { return err.Error() } @@ -657,17 +657,17 @@ func TestDataDriven(t *testing.T) { var setting, value string d.ScanArgs(t, "setting", &setting) d.ScanArgs(t, "value", &value) - server := lastCreatedServer - serverutils.SetClusterSetting(t, ds.clusters[server], setting, value) + cluster := lastCreatedCluster + serverutils.SetClusterSetting(t, ds.clusters[cluster], setting, value) return "" case "let": - server := lastCreatedServer + cluster := lastCreatedCluster user := "root" if len(d.CmdArgs) == 0 { t.Fatalf("Must specify at least one variable name.") } - rows, err := ds.getSQLDB(t, server, user).Query(d.Input) + rows, err := ds.getSQLDB(t, cluster, user).Query(d.Input) if err != nil { return err.Error() } @@ -731,7 +731,7 @@ func TestDataDriven(t *testing.T) { return execWithTagAndPausePoint(jobspb.TypeSchemaChange) case "job": - server := lastCreatedServer + cluster := lastCreatedCluster const user = "root" if d.HasArg("cancel") { @@ -742,7 +742,7 @@ func TestDataDriven(t *testing.T) { if jobID, ok = ds.jobTags[cancelJobTag]; !ok { t.Fatalf("could not find job with tag %s", cancelJobTag) } - runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, server, user)) + runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, cluster, user)) runner.Exec(t, `CANCEL JOB $1`, jobID) jobutils.WaitForJobToCancel(t, runner, jobID) } else if d.HasArg("resume") { @@ -753,7 +753,7 @@ func TestDataDriven(t *testing.T) { if jobID, ok = ds.jobTags[resumeJobTag]; !ok { t.Fatalf("could not find job with tag %s", resumeJobTag) } - runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, server, user)) + runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, cluster, user)) runner.Exec(t, `RESUME JOB $1`, jobID) } else if d.HasArg("wait-for-state") { var tag string @@ -763,7 +763,7 @@ func TestDataDriven(t *testing.T) { if jobID, ok = ds.jobTags[tag]; !ok { t.Fatalf("could not find job with tag %s", tag) } - runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, server, user)) + runner := sqlutils.MakeSQLRunner(ds.getSQLDB(t, cluster, user)) var state string d.ScanArgs(t, "wait-for-state", &state) switch state { @@ -789,11 +789,11 @@ func TestDataDriven(t *testing.T) { var target string d.ScanArgs(t, "target", &target) - handleKVRequest(ctx, t, lastCreatedServer, ds, request, target) + handleKVRequest(ctx, t, lastCreatedCluster, ds, request, target) return "" case "save-cluster-ts": - server := lastCreatedServer + cluster := lastCreatedCluster const user = "root" var timestampTag string d.ScanArgs(t, "tag", ×tampTag) @@ -801,15 +801,15 @@ func TestDataDriven(t *testing.T) { t.Fatalf("cannot reuse cluster ts tag %s", timestampTag) } var ts string - err := ds.getSQLDB(t, server, user).QueryRow(`SELECT cluster_logical_timestamp()`).Scan(&ts) + err := ds.getSQLDB(t, cluster, user).QueryRow(`SELECT cluster_logical_timestamp()`).Scan(&ts) require.NoError(t, err) ds.clusterTimestamps[timestampTag] = ts return "" case "create-dummy-system-table": - db := ds.servers[lastCreatedServer].DB() - execCfg := ds.servers[lastCreatedServer].ExecutorConfig().(sql.ExecutorConfig) - testTenants := ds.servers[lastCreatedServer].TestTenants() + db := ds.firstNode[lastCreatedCluster].DB() + execCfg := ds.firstNode[lastCreatedCluster].ExecutorConfig().(sql.ExecutorConfig) + testTenants := ds.firstNode[lastCreatedCluster].TestTenants() if len(testTenants) > 0 { execCfg = testTenants[0].ExecutorConfig().(sql.ExecutorConfig) } @@ -833,7 +833,7 @@ func TestDataDriven(t *testing.T) { return "" case "corrupt-backup": - server := lastCreatedServer + cluster := lastCreatedCluster const user = "root" var uri string d.ScanArgs(t, "uri", &uri) @@ -841,9 +841,9 @@ func TestDataDriven(t *testing.T) { require.NoError(t, err) var filePath string filePathQuery := fmt.Sprintf("SELECT path FROM [SHOW BACKUP FILES FROM LATEST IN %s] LIMIT 1", uri) - err = ds.getSQLDB(t, server, user).QueryRow(filePathQuery).Scan(&filePath) + err = ds.getSQLDB(t, cluster, user).QueryRow(filePathQuery).Scan(&filePath) require.NoError(t, err) - fullPath := filepath.Join(ds.getIODir(t, server), parsedURI.Path, filePath) + fullPath := filepath.Join(ds.getIODir(t, cluster), parsedURI.Path, filePath) print(fullPath) data, err := os.ReadFile(fullPath) require.NoError(t, err) @@ -853,11 +853,11 @@ func TestDataDriven(t *testing.T) { } return "" case "link-backup": - server := lastCreatedServer + cluster := lastCreatedCluster sourceRelativePath := "" destRelativePath := "" - ioDir := ds.getIODir(t, server) - d.ScanArgs(t, "server", &server) + ioDir := ds.getIODir(t, cluster) + d.ScanArgs(t, "cluster", &cluster) d.ScanArgs(t, "src-path", &sourceRelativePath) d.ScanArgs(t, "dest-path", &destRelativePath) splitSrcPath := strings.Split(sourceRelativePath, ",") @@ -876,12 +876,12 @@ func TestDataDriven(t *testing.T) { } func handleKVRequest( - ctx context.Context, t *testing.T, server string, ds datadrivenTestState, request, target string, + ctx context.Context, t *testing.T, cluster string, ds datadrivenTestState, request, target string, ) { user := "root" if request == "DeleteRange" { var tableID uint32 - err := ds.getSQLDB(t, server, user).QueryRow(`SELECT id FROM system.namespace WHERE name = $1`, + err := ds.getSQLDB(t, cluster, user).QueryRow(`SELECT id FROM system.namespace WHERE name = $1`, target).Scan(&tableID) require.NoError(t, err) bankSpan := makeTableSpan(keys.SystemSQLCodec, tableID) @@ -893,7 +893,7 @@ func handleKVRequest( }, UseRangeTombstone: true, } - if _, err := kv.SendWrapped(ctx, ds.servers[server].DistSenderI().(*kvcoord.DistSender), &dr); err != nil { + if _, err := kv.SendWrapped(ctx, ds.firstNode[cluster].DistSenderI().(*kvcoord.DistSender), &dr); err != nil { t.Fatal(err) } } else { diff --git a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/backup-options b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/backup-options index 1babcaa762cb..c6405c144611 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/backup-options +++ b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/backup-options @@ -1,4 +1,4 @@ -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- # Create test schedules. diff --git a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/missing-schedule b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/missing-schedule index 650f590eb874..3f388b9e0b8b 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/missing-schedule +++ b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/missing-schedule @@ -1,4 +1,4 @@ -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- # Test that dropping the full backup causes a graceful failure on ALTER. diff --git a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/recurrence b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/recurrence index b3d4b843f553..eac1f9529bdf 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/recurrence +++ b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/recurrence @@ -1,4 +1,4 @@ -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- # Create test schedules. diff --git a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options index 168958cfcf3a..c7f441be09ba 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options +++ b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options @@ -1,4 +1,4 @@ -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- # Create test schedules. diff --git a/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors b/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors index 486b754f4f2e..c7cc86fd2228 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors +++ b/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors @@ -2,7 +2,7 @@ # and type descriptors in the DROP state. subtest dropped-database-descriptors -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -105,7 +105,7 @@ subtest end # Test backup/restore interaction with dropped schema and type in a database. subtest dropped-schema-descriptors -new-server name=s2 +new-cluster name=s2 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors-declarative b/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors-declarative index 2890ed938e41..c2cb121ca4a9 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors-declarative +++ b/pkg/ccl/backupccl/testdata/backup-restore/backup-dropped-descriptors-declarative @@ -2,7 +2,7 @@ # and type descriptors in the DROP state. subtest dropped-database-descriptors -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -58,7 +58,7 @@ subtest end # Test backup/restore interaction with dropped schema and type in a database. subtest dropped-schema-descriptors -new-server name=s2 +new-cluster name=s2 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions b/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions index 387616d17ef4..39d4877f19c3 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions +++ b/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions-deprecated b/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions-deprecated index cd4ed4d89986..6a9dd511e307 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions-deprecated +++ b/pkg/ccl/backupccl/testdata/backup-restore/backup-permissions-deprecated @@ -1,5 +1,5 @@ # Test permissions checks for non-admin users running BACKUP. -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -42,7 +42,7 @@ BACKUP TABLE d.t INTO 'nodelocal://0/test-nonroot-table'; # Start a new cluster with the same IO dir. # nodelocal IO is a form of implicit access (since it is accessed as the node, # so we require the allow-implicit-access flag. -new-server name=s2 allow-implicit-access +new-cluster name=s2 allow-implicit-access ---- exec-sql @@ -121,18 +121,18 @@ GRANT USAGE ON TYPE d2.greeting TO testuser; ---- # testuser should now have all the required privileges. -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser BACKUP DATABASE d2 INTO 'nodelocal://0/d2'; ---- NOTICE: The existing privileges are being deprecated in favour of a fine-grained privilege model explained here . In a future release, to run BACKUP DATABASE, user testuser will exclusively require the BACKUP privilege on database d2. -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser BACKUP TABLE d2.t INTO 'nodelocal://0/d2-table'; ---- NOTICE: The existing privileges are being deprecated in favour of a fine-grained privilege model explained here . In a future release, to run BACKUP TABLE, user testuser will exclusively require the BACKUP privilege on tables: t. # Test that implicit access is disallowed when the testing knob is not set. -new-server name=s3 share-io-dir=s1 +new-cluster name=s3 share-io-dir=s1 ---- exec-sql @@ -142,7 +142,7 @@ CREATE USER testuser; GRANT CONNECT ON DATABASE d TO testuser; ---- -exec-sql server=s3 user=testuser +exec-sql cluster=s3 user=testuser SHOW BACKUP 'http://COCKROACH_TEST_HTTP_SERVER/' ---- pq: only users with the admin role or the EXTERNALIOIMPLICITACCESS system privilege are allowed to access the specified http URI @@ -153,14 +153,14 @@ BACKUP DATABASE d INTO 'nodelocal://0/test3' pq: only users with the admin role or the EXTERNALIOIMPLICITACCESS system privilege are allowed to access the specified nodelocal URI # Test that http access is disallowed by disable http even if allow-non-admin is on. -new-server name=s4 allow-implicit-access disable-http +new-cluster name=s4 allow-implicit-access disable-http ---- -exec-sql server=s4 +exec-sql cluster=s4 CREATE USER testuser ---- -exec-sql server=s4 user=testuser +exec-sql cluster=s4 user=testuser SHOW BACKUP 'http://COCKROACH_TEST_HTTP_SERVER/' ---- pq: make storage: external http access disabled diff --git a/pkg/ccl/backupccl/testdata/backup-restore/column-families b/pkg/ccl/backupccl/testdata/backup-restore/column-families index 16d909dca6bc..9861fb8b1c22 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/column-families +++ b/pkg/ccl/backupccl/testdata/backup-restore/column-families @@ -1,6 +1,6 @@ # disabled to run within tenant because ALTER SPLIT cmd is not supported within tenant -new-server name=s1 disable-tenant localities=us-east-1,us-west-1,us-west-2,eu-central-1 +new-cluster name=s1 disable-tenant localities=us-east-1,us-west-1,us-west-2,eu-central-1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/descriptor-broadening b/pkg/ccl/backupccl/testdata/backup-restore/descriptor-broadening index 11553de952e0..511b706e923e 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/descriptor-broadening +++ b/pkg/ccl/backupccl/testdata/backup-restore/descriptor-broadening @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/descriptor-conflicts b/pkg/ccl/backupccl/testdata/backup-restore/descriptor-conflicts index a2feb380068b..6cb6cdcd0a47 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/descriptor-conflicts +++ b/pkg/ccl/backupccl/testdata/backup-restore/descriptor-conflicts @@ -7,7 +7,7 @@ # disabled to run within a tenant because they cannot set zone configs # https://github.com/cockroachdb/cockroach/issues/49854?version=v22.2 -new-server name=s1 disable-tenant +new-cluster name=s1 disable-tenant ---- exec-sql @@ -26,7 +26,7 @@ exec-sql BACKUP INTO 'nodelocal://0/conflicting-descriptors'; ---- -new-server name=s2 share-io-dir=s1 disable-tenant +new-cluster name=s2 share-io-dir=s1 disable-tenant ---- # Create 4 dummy system tables that will have conflicting IDs with the database, diff --git a/pkg/ccl/backupccl/testdata/backup-restore/drop-schedule-backup b/pkg/ccl/backupccl/testdata/backup-restore/drop-schedule-backup index eb31523c5789..12e904ef7b2e 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/drop-schedule-backup +++ b/pkg/ccl/backupccl/testdata/backup-restore/drop-schedule-backup @@ -1,7 +1,7 @@ # These tests validate the DROP SCHEDULE command for dropping the incremental scheduled # backup when the full scheduled backup is dropped. -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/encrypted-backups b/pkg/ccl/backupccl/testdata/backup-restore/encrypted-backups index 1c7e68dcaee3..4984cd1d9525 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/encrypted-backups +++ b/pkg/ccl/backupccl/testdata/backup-restore/encrypted-backups @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/external-connections-nodelocal b/pkg/ccl/backupccl/testdata/backup-restore/external-connections-nodelocal index 486cc7292c73..3ae38729a188 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/external-connections-nodelocal +++ b/pkg/ccl/backupccl/testdata/backup-restore/external-connections-nodelocal @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- subtest basic-backup-nodelocal @@ -99,7 +99,7 @@ subtest end subtest basic-restore-nodelocal -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- # Cluster restore. @@ -157,7 +157,7 @@ subtest end subtest incremental-location-backup-restore-nodelocal -switch-server name=s1 +switch-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/external-connections-privileges b/pkg/ccl/backupccl/testdata/backup-restore/external-connections-privileges index e27a5d3e847b..591833995b80 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/external-connections-privileges +++ b/pkg/ccl/backupccl/testdata/backup-restore/external-connections-privileges @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- subtest backup-restore-privileges diff --git a/pkg/ccl/backupccl/testdata/backup-restore/external-connections-userfile b/pkg/ccl/backupccl/testdata/backup-restore/external-connections-userfile index ef7f7dd154e0..0b42bd162f15 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/external-connections-userfile +++ b/pkg/ccl/backupccl/testdata/backup-restore/external-connections-userfile @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- subtest basic-backup-userfile @@ -136,7 +136,7 @@ subtest end subtest incremental-location-backup-restore-userfile -switch-server name=s1 +switch-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/feature-flags b/pkg/ccl/backupccl/testdata/backup-restore/feature-flags index a7cee25a41c5..a3e28b4dfa75 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/feature-flags +++ b/pkg/ccl/backupccl/testdata/backup-restore/feature-flags @@ -2,7 +2,7 @@ subtest backup-feature-flags # disabled for tenants as they can't enable/disable backup and restore features -new-server name=s1 disable-tenant +new-cluster name=s1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write b/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write index 328263642493..fa7f38faa5b2 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write +++ b/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write @@ -1,6 +1,6 @@ subtest backup_file_table -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/import-start-time b/pkg/ccl/backupccl/testdata/backup-restore/import-start-time index df9e0167b3ed..1ade5232298f 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/import-start-time +++ b/pkg/ccl/backupccl/testdata/backup-restore/import-start-time @@ -5,7 +5,7 @@ ## -check that the ImportStartTime is set on the descriptor ## -check that it's removed after cancellation / success -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-missing b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-missing index ddd402d0c567..94b3e1a747b3 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-missing +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-missing @@ -22,25 +22,25 @@ # - revision history cluster backups # - revision history table/database backups which includes an incremental backup between t1 and t2 -new-server name=s1 allow-implicit-access disable-tenant +new-cluster name=s1 allow-implicit-access disable-tenant ---- # Link 4 backup chains: # 1. A corrupt database backup chain with revision history -link-backup server=s1 src-path=restore_importing_tables,database dest-path=database +link-backup cluster=s1 src-path=restore_importing_tables,database dest-path=database ---- # 2. A clean database backup chain with revision history which includes two # backups that observe the offline descriptors. -link-backup server=s1 src-path=restore_importing_tables,database_double_inc dest-path=database_double_inc +link-backup cluster=s1 src-path=restore_importing_tables,database_double_inc dest-path=database_double_inc ---- # 3. A clean data database backup chain without revision history -link-backup server=s1 src-path=restore_importing_tables,database_no_hist dest-path=database_no_hist +link-backup cluster=s1 src-path=restore_importing_tables,database_no_hist dest-path=database_no_hist ---- # 4. A clean cluster backup chain with revision history -link-backup server=s1 src-path=restore_importing_tables,cluster dest-path=cluster +link-backup cluster=s1 src-path=restore_importing_tables,cluster dest-path=cluster ---- # Note that we're purposely skipping the reintroduction of foo, foofoo, goodfoo in the @@ -212,7 +212,7 @@ SELECT count(*) FROM d.goodfoo; # Check the cluster level restore -new-server name=s2 share-io-dir=s1 allow-implicit-access disable-tenant +new-cluster name=s2 share-io-dir=s1 allow-implicit-access disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback index fc873c1497f5..5cd89c3e29ab 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-import-rollback @@ -9,7 +9,7 @@ # disabled to run within tenant as they don't have access to the # storage.mvcc.range_tombstones.enabled cluster setting -new-server name=s1 disable-tenant +new-cluster name=s1 disable-tenant ---- ########### diff --git a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports index 828c3b724ade..3bb53b973180 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports @@ -5,7 +5,7 @@ # On an unfinalized cluster: a backed up import should not get restored -new-server name=s1 +new-cluster name=s1 ---- @@ -182,7 +182,7 @@ d foofoo table 3 incremental # Ensure all the RESTOREs contain foo (no data) and foofoo (1 row) as of system time t0 -new-server name=s2 share-io-dir=s1 allow-implicit-access +new-cluster name=s2 share-io-dir=s1 allow-implicit-access ---- @@ -246,7 +246,7 @@ SELECT * FROM d.foofoo; # Ensure the imported data exists as of latest time -new-server name=s3 share-io-dir=s1 allow-implicit-access +new-cluster name=s3 share-io-dir=s1 allow-implicit-access ---- @@ -334,7 +334,7 @@ SELECT * FROM d.foofoo; # More investigation required. -new-server name=s4 share-io-dir=s1 allow-implicit-access beforeVersion=Start22_2 disable-tenant +new-cluster name=s4 share-io-dir=s1 allow-implicit-access beforeVersion=Start22_2 disable-tenant ---- exec-sql @@ -508,7 +508,7 @@ d foo table 2 incremental d foofoo table 3 incremental -upgrade-server version=Start22_2 +upgrade-cluster version=Start22_2 ---- exec-sql @@ -532,7 +532,7 @@ d foofoo table 3 incremental # Restore the backups taken from a mixed version chain -new-server name=s5 share-io-dir=s1 allow-implicit-access disable-tenant +new-cluster name=s5 share-io-dir=s1 allow-implicit-access disable-tenant ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-restores b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-restores index 9b87734036e6..4e4e2db187cc 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-restores +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-restores @@ -16,7 +16,7 @@ # offline tables. -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -116,7 +116,7 @@ b2 baz table 3 incremental # Ensure the restored cluster contains nothing from the in-progress restoring database as of system # time t0 -new-server name=s2 share-io-dir=s1 allow-implicit-access +new-cluster name=s2 share-io-dir=s1 allow-implicit-access ---- @@ -131,7 +131,7 @@ SELECT database_name FROM [SHOW DATABASES] WHERE database_name = 'b2'; # Ensure restored cluster contains the restored database as of latest time -new-server name=s3 share-io-dir=s1 allow-implicit-access +new-cluster name=s3 share-io-dir=s1 allow-implicit-access ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/lock-concurrent-backups b/pkg/ccl/backupccl/testdata/backup-restore/lock-concurrent-backups index 671a7bb9cc98..bd9cf95217ca 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/lock-concurrent-backups +++ b/pkg/ccl/backupccl/testdata/backup-restore/lock-concurrent-backups @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- # Test that a backup job does not read its own lock file on resumption, diff --git a/pkg/ccl/backupccl/testdata/backup-restore/max-row-size b/pkg/ccl/backupccl/testdata/backup-restore/max-row-size index a1a587c49718..ecb3d9ebdf5a 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/max-row-size +++ b/pkg/ccl/backupccl/testdata/backup-restore/max-row-size @@ -1,4 +1,4 @@ -new-server name=m1 +new-cluster name=m1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/mismatched-localities b/pkg/ccl/backupccl/testdata/backup-restore/mismatched-localities index 7c8d76989d2e..318d01e22be3 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/mismatched-localities +++ b/pkg/ccl/backupccl/testdata/backup-restore/mismatched-localities @@ -1,6 +1,6 @@ # disabled to run within tenant because multiregion primitives are not supported within tenant -new-server name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1 +new-cluster name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/multiregion b/pkg/ccl/backupccl/testdata/backup-restore/multiregion index 4f5f5013a7e9..a9a769641274 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/multiregion +++ b/pkg/ccl/backupccl/testdata/backup-restore/multiregion @@ -1,6 +1,6 @@ # disabled to run within tenant because multiregion primitives are not supported within tenant -new-server name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 +new-cluster name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 ---- exec-sql @@ -25,7 +25,7 @@ BACKUP INTO 'nodelocal://1/full_cluster_backup/'; ---- # A new cluster with the same locality settings. -new-server name=s2 share-io-dir=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 +new-cluster name=s2 share-io-dir=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 ---- exec-sql @@ -50,7 +50,7 @@ postgres root {} system node {} # A new cluster with different localities settings. -new-server name=s3 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 +new-cluster name=s3 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 ---- exec-sql @@ -140,7 +140,7 @@ BACKUP DATABASE eu_central_db INTO 'nodelocal://1/eu_central_database_backup/'; ---- # New cluster for a cluster backup. -new-server name=s4 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 +new-cluster name=s4 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 ---- set-cluster-setting setting=sql.defaults.primary_region value=eu-north-1 diff --git a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys index 4163bf7698b4..89f34540b321 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys +++ b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys @@ -3,7 +3,7 @@ # disabled to run within tenants because the kv request cmd only works on system tenants -new-server name=s1 disable-tenant +new-cluster name=s1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys-revision-history b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys-revision-history index 200675ebdb46..7629d1d01b25 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys-revision-history +++ b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys-revision-history @@ -11,7 +11,7 @@ # disabled to run within tenants because the kv request cmd only works on system tenants -new-server name=s1 disable-tenant +new-cluster name=s1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-check-descriptors b/pkg/ccl/backupccl/testdata/backup-restore/restore-check-descriptors index e8feeca64709..455efa462632 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-check-descriptors +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-check-descriptors @@ -4,7 +4,7 @@ # cannot fix this specific corrupt descriptor problem automatically) and does # not publish restoring database with the corrupt descriptor. -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-grants b/pkg/ccl/backupccl/testdata/backup-restore/restore-grants index 43181a2f7c7e..de4d7e53f2d1 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-grants +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-grants @@ -9,7 +9,7 @@ # We allow implicit access to non-admin users so that we can test # with nodelocal. -new-server name=s1 allow-implicit-access disable-tenant +new-cluster name=s1 allow-implicit-access disable-tenant ---- # TODO(ssd): We reset the closed timestamp configurables to avoid schema @@ -163,7 +163,7 @@ BACKUP INTO 'nodelocal://0/test/' # above. subtest cluster-restore -new-server name=s2 share-io-dir=s1 allow-implicit-access disable-tenant +new-cluster name=s2 share-io-dir=s1 allow-implicit-access disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-mixed-version b/pkg/ccl/backupccl/testdata/backup-restore/restore-mixed-version index 1385a2f8de43..396d2d60b58b 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-mixed-version +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-mixed-version @@ -1,4 +1,4 @@ -new-server name=s1 beforeVersion=Start22_2 disable-tenant +new-cluster name=s1 beforeVersion=Start22_2 disable-tenant ---- exec-sql @@ -12,13 +12,13 @@ exec-sql BACKUP INTO 'nodelocal://1/full_cluster_backup/'; ---- -# This is a server where the cluster version is behind the binary version. Such +# This is a cluster where the cluster version is behind the binary version. Such # a condition only occurs when the user has upgraded the node to a new major # version but has not yet finalized the upgrade. -new-server name=s2 beforeVersion=Start22_2 share-io-dir=s1 disable-tenant +new-cluster name=s2 beforeVersion=Start22_2 share-io-dir=s1 disable-tenant ---- -exec-sql expect-error-regex=pq: cluster restore not supported during major version upgrade: restore started at cluster version 22.1 but binary version is.* +exec-sql expect-error-regex=(pq: cluster restore not supported during major version upgrade: restore started at cluster version .* but binary version is.*) RESTORE FROM LATEST IN 'nodelocal://1/full_cluster_backup/' ---- regex matches error diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop index a52d5ba15c18..69acb8222eb2 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-fast-drop @@ -1,4 +1,4 @@ -new-server name=s1 nodes=1 disable-tenant +new-cluster name=s1 nodes=1 disable-tenant ---- subtest restore-cleanup-with-range-tombstones @@ -16,7 +16,7 @@ exec-sql BACKUP INTO 'nodelocal://1/cluster_backup'; ---- -new-server name=s2 nodes=1 share-io-dir=s1 disable-tenant +new-cluster name=s2 nodes=1 share-io-dir=s1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry index b8daff5a057c..14a927808ba6 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-retry @@ -1,5 +1,5 @@ # disabled for tenants as they can't set storage.mvcc.range_tombstones.enabled -new-server name=s1 nodes=1 disable-tenant +new-cluster name=s1 nodes=1 disable-tenant ---- subtest restore-retry @@ -17,7 +17,7 @@ exec-sql BACKUP INTO 'nodelocal://1/cluster_backup'; ---- -new-server name=s2 nodes=1 share-io-dir=s1 disable-tenant +new-cluster name=s2 nodes=1 share-io-dir=s1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-schema-objects b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-schema-objects index 40f1ad2b4404..179474a13ed2 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-schema-objects +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-schema-objects @@ -1,4 +1,4 @@ -new-server name=s1 nodes=1 splits=1000 +new-cluster name=s1 nodes=1 splits=1000 ---- subtest restore-cleanup @@ -148,7 +148,7 @@ BACKUP INTO 'nodelocal://1/cluster'; ---- # Start a new cluster with the same IO dir. -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- # We pause the job before publishing descriptors to ensure that we are testing diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-ttl b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-ttl index d53a527a5d60..89b29fdb1d3a 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-ttl +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-on-fail-or-cancel-ttl @@ -1,7 +1,7 @@ # This test ensures that a failed or cancelled restore cleans up the ttl schedule # that is created when resotring a table with a row level ttl configuration. -new-server name=s1 nodes=1 +new-cluster name=s1 nodes=1 ---- subtest restore-fail-cleans-up-ttl-schedules-before-publishing diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions b/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions index cc7a9c2d51ac..6f6c407d4ddc 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -177,7 +177,7 @@ REVOKE RESTORE ON DATABASE nonadmin FROM testuser; subtest end subtest cluster-restore -new-server name=s2 allow-implicit-access share-io-dir=s1 +new-cluster name=s2 allow-implicit-access share-io-dir=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions-deprecated b/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions-deprecated index 2ef091503933..2d467710fd1a 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions-deprecated +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-permissions-deprecated @@ -1,5 +1,5 @@ # Test permissions checks for non-admin users running RESTORE. -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -27,65 +27,65 @@ RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/test/'; ---- # Start a new cluster with the same IO dir. -new-server name=s2 share-io-dir=s1 allow-implicit-access +new-cluster name=s2 share-io-dir=s1 allow-implicit-access ---- -exec-sql server=s2 +exec-sql cluster=s2 CREATE USER testuser ---- # Restore into the new cluster. -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser RESTORE FROM LATEST IN 'nodelocal://0/test/' ---- pq: only users with the admin role or the RESTORE system privilege are allowed to perform a cluster restore -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/test/' ---- pq: only users with the CREATEDB privilege can restore databases HINT: The existing privileges are being deprecated in favour of a fine-grained privilege model explained here . In a future release, to run RESTORE DATABASE, user testuser will exclusively require the RESTORE system privilege. -exec-sql server=s2 +exec-sql cluster=s2 CREATE DATABASE d ---- -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser RESTORE TABLE d.t FROM LATEST IN 'nodelocal://0/test/' ---- pq: user testuser does not have CREATE privilege on database d HINT: The existing privileges are being deprecated in favour of a fine-grained privilege model explained here . In a future release, to run RESTORE TABLE, user testuser will exclusively require the RESTORE privilege on database d. -exec-sql server=s2 +exec-sql cluster=s2 GRANT CREATE ON DATABASE d TO testuser ---- -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser RESTORE TABLE d.t FROM LATEST IN 'nodelocal://0/test/' ---- NOTICE: The existing privileges are being deprecated in favour of a fine-grained privilege model explained here . In a future release, to run RESTORE TABLE, user testuser will exclusively require the RESTORE privilege on databases d -query-sql server=s2 +query-sql cluster=s2 SELECT x FROM d.t ORDER BY x ---- 1 2 3 -exec-sql server=s2 +exec-sql cluster=s2 DROP DATABASE d ---- -exec-sql server=s2 +exec-sql cluster=s2 ALTER USER testuser CREATEDB ---- -exec-sql server=s2 user=testuser +exec-sql cluster=s2 user=testuser RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/test/' ---- NOTICE: The existing privileges are being deprecated in favour of a fine-grained privilege model explained here . In a future release, to run RESTORE DATABASE, user testuser will exclusively require the RESTORE system privilege. -query-sql server=s2 +query-sql cluster=s2 SELECT x FROM d.t ORDER BY x ---- 1 @@ -93,14 +93,14 @@ SELECT x FROM d.t ORDER BY x 3 # Test that implicit access is disallowed when the testing knob isn't set. -new-server name=s3 share-io-dir=s1 +new-cluster name=s3 share-io-dir=s1 ---- -exec-sql server=s3 +exec-sql cluster=s3 CREATE USER testuser ---- -exec-sql server=s3 user=testuser +exec-sql cluster=s3 user=testuser RESTORE TABLE d.t FROM LATEST IN 'nodelocal://0/test/' ---- pq: only users with the admin role or the EXTERNALIOIMPLICITACCESS system privilege are allowed to access the specified nodelocal URI diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only index f00309fda338..6a1ff033076d 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only @@ -1,6 +1,6 @@ # Test schema_only restore -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- exec-sql @@ -51,7 +51,7 @@ BACKUP Database d INTO 'nodelocal://1/full_database_backup/'; # A new cluster in prep for a cluster level schema_only restore. -new-server name=s2 share-io-dir=s1 allow-implicit-access +new-cluster name=s2 share-io-dir=s1 allow-implicit-access ---- # First, ensure cluster level schema_only restore fails fast in same ways as a cluster level restore. @@ -63,7 +63,7 @@ RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only, ne pq: new_db_name can only be used for RESTORE DATABASE with a single target database -exec-sql server=s2 +exec-sql cluster=s2 CREATE USER testuser ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-mixed-version b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-mixed-version index b9b486b8afba..4c5e66f483d9 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-mixed-version +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-mixed-version @@ -2,7 +2,7 @@ # Disabled to run within tenants because version gating infra does not play nice within tenants. # More investigation required. -new-server name=s1 beforeVersion=Start22_2 disable-tenant +new-cluster name=s1 beforeVersion=Start22_2 disable-tenant ---- exec-sql @@ -21,7 +21,7 @@ RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/full_database_backup/' with sch ---- pq: cannot run RESTORE with schema_only until cluster has fully upgraded to 22.2 -upgrade-server version=Start22_2 +upgrade-cluster version=Start22_2 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion index 5de89249c0da..a972dcd67bf3 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-schema-only-multiregion @@ -3,7 +3,7 @@ # disabled to run within tenant because multiregion primitives are not supported within tenant -new-server name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 +new-cluster name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 ---- exec-sql @@ -28,7 +28,7 @@ BACKUP INTO 'nodelocal://1/full_cluster_backup/'; ---- # A new cluster with the same locality settings. -new-server name=s2 share-io-dir=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 +new-cluster name=s2 share-io-dir=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1 ---- exec-sql @@ -53,7 +53,7 @@ postgres root {} system node {} # A new cluster with different localities settings. -new-server name=s3 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 +new-cluster name=s3 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 ---- exec-sql @@ -143,7 +143,7 @@ BACKUP DATABASE eu_central_db INTO 'nodelocal://1/eu_central_database_backup/'; ---- # New cluster for a cluster backup. -new-server name=s4 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 +new-cluster name=s4 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1 ---- set-cluster-setting setting=sql.defaults.primary_region value=eu-north-1 diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants b/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants index c0f20e7ebe39..b64e0bdd5d11 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants @@ -1,7 +1,7 @@ # disabled to probabilistically run within a tenant because the test always runs from the host # tenant -new-server name=s1 disable-tenant +new-cluster name=s1 disable-tenant ---- # Create a few tenants. @@ -38,7 +38,7 @@ exec-sql BACKUP TENANT 6 INTO 'nodelocal://1/tenant6' ---- -new-server name=s2 share-io-dir=s1 disable-tenant +new-cluster name=s2 share-io-dir=s1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-validation-only b/pkg/ccl/backupccl/testdata/backup-restore/restore-validation-only index 0d4efbc86ab0..6a31df08026b 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-validation-only +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-validation-only @@ -6,7 +6,7 @@ # Part 1 ########## -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- exec-sql @@ -57,7 +57,7 @@ BACKUP Database d INTO 'nodelocal://1/full_database_backup/'; # A new cluster in prep for a cluster level schema_only restore. -new-server name=s2 share-io-dir=s1 allow-implicit-access +new-cluster name=s2 share-io-dir=s1 allow-implicit-access ---- # First, ensure cluster level schema_only restore fails fast in same ways as a cluster level restore. @@ -69,7 +69,7 @@ RESTORE FROM LATEST IN 'nodelocal://0/full_cluster_backup/' with schema_only, ve pq: new_db_name can only be used for RESTORE DATABASE with a single target database -exec-sql server=s2 +exec-sql cluster=s2 CREATE USER testuser ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/revision_history b/pkg/ccl/backupccl/testdata/backup-restore/revision_history index f003a5a62ab7..4925c2cec3f5 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/revision_history +++ b/pkg/ccl/backupccl/testdata/backup-restore/revision_history @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- # Regression test for #62738. diff --git a/pkg/ccl/backupccl/testdata/backup-restore/row_level_ttl b/pkg/ccl/backupccl/testdata/backup-restore/row_level_ttl index 4fda2b7406ab..225ca93edaff 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/row_level_ttl +++ b/pkg/ccl/backupccl/testdata/backup-restore/row_level_ttl @@ -1,4 +1,4 @@ -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- exec-sql @@ -15,7 +15,7 @@ exec-sql BACKUP INTO 'nodelocal://1/full_cluster_backup/' ---- -new-server name=s2 share-io-dir=s1 allow-implicit-access +new-cluster name=s2 share-io-dir=s1 allow-implicit-access ---- query-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges b/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges index fb92667f8454..87678b67c0a6 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges +++ b/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- subtest create-scheduled-privileges diff --git a/pkg/ccl/backupccl/testdata/backup-restore/show-backup-multiregion b/pkg/ccl/backupccl/testdata/backup-restore/show-backup-multiregion index cf4f446be435..49b089711264 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/show-backup-multiregion +++ b/pkg/ccl/backupccl/testdata/backup-restore/show-backup-multiregion @@ -1,6 +1,6 @@ # These tests validate the SHOW BACKUP command for multi-region databases. -new-server name=s1 allow-implicit-access localities=us-east-1,us-west-1,eu-central-1 disable-tenant +new-cluster name=s1 allow-implicit-access localities=us-east-1,us-west-1,eu-central-1 disable-tenant ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/show-backup-union b/pkg/ccl/backupccl/testdata/backup-restore/show-backup-union index 5b70d4e6a752..aa73a7889823 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/show-backup-union +++ b/pkg/ccl/backupccl/testdata/backup-restore/show-backup-union @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/show-schedules-old b/pkg/ccl/backupccl/testdata/backup-restore/show-schedules-old index f04fe366229c..3ddb9bd53b85 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/show-schedules-old +++ b/pkg/ccl/backupccl/testdata/backup-restore/show-schedules-old @@ -2,10 +2,10 @@ # versions to ensure that they can unmarshalled correctly in the current # version. -new-server name=s1 +new-cluster name=s1 ---- -link-backup server=s1 src-path=show_schedules_old,system_scheduled_jobs_22_1 dest-path=system_scheduled_jobs_22_1 +link-backup cluster=s1 src-path=show_schedules_old,system_scheduled_jobs_22_1 dest-path=system_scheduled_jobs_22_1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/show_backup b/pkg/ccl/backupccl/testdata/backup-restore/show_backup index d32c318b7d58..041af03e5392 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/show_backup +++ b/pkg/ccl/backupccl/testdata/backup-restore/show_backup @@ -1,10 +1,10 @@ # These tests validate the SHOW BACKUP command (old and new stynax) with # backup images that contain both invalid and valid sets of descriptors. -new-server name=s1 allow-implicit-access +new-cluster name=s1 allow-implicit-access ---- -link-backup server=s1 src-path=show_backup_validate,invalidDependOnBy_21.1 dest-path=invalidDependOnBy_21.1 +link-backup cluster=s1 src-path=show_backup_validate,invalidDependOnBy_21.1 dest-path=invalidDependOnBy_21.1 ---- # This backup intentionally has a dangling invalid depend on by reference. @@ -13,7 +13,7 @@ SELECT * FROM [SHOW BACKUP VALIDATE FROM 'invalidDependOnBy_21.1' IN 'nodelocal: ---- true -link-backup server=s1 src-path=show_backup_validate,valid-22.2 dest-path=valid-22.2 +link-backup cluster=s1 src-path=show_backup_validate,valid-22.2 dest-path=valid-22.2 ---- # This backup is completely valid, but has no jobs. @@ -22,7 +22,7 @@ SELECT * FROM [SHOW BACKUP VALIDATE FROM 'valid-22.2' IN 'nodelocal://0/']; ---- true -link-backup server=s1 src-path=show_backup_validate,valid-22.2-with-job dest-path=valid-22.2-with-job +link-backup cluster=s1 src-path=show_backup_validate,valid-22.2-with-job dest-path=valid-22.2-with-job ---- # This back up is valid, and taken when a job was actively working on the diff --git a/pkg/ccl/backupccl/testdata/backup-restore/system-privileges-table b/pkg/ccl/backupccl/testdata/backup-restore/system-privileges-table index 4589f30f9ebd..cbf8a7a3affc 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/system-privileges-table +++ b/pkg/ccl/backupccl/testdata/backup-restore/system-privileges-table @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -29,15 +29,15 @@ BACKUP INTO 'nodelocal://0/test/' ---- # Start a new cluster with the same IO dir. -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- # Restore into the new cluster. -exec-sql server=s2 +exec-sql cluster=s2 RESTORE FROM LATEST IN 'nodelocal://0/test/' ---- -query-sql server=s2 +query-sql cluster=s2 SELECT * FROM system.privileges ---- root /externalconn/foo {ALL} {} diff --git a/pkg/ccl/backupccl/testdata/backup-restore/temp-tables b/pkg/ccl/backupccl/testdata/backup-restore/temp-tables index e8695f5f2212..c6ea28b5729d 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/temp-tables +++ b/pkg/ccl/backupccl/testdata/backup-restore/temp-tables @@ -1,6 +1,6 @@ # Test that temporary tables do not show up in any backup. -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -112,7 +112,7 @@ SELECT table_name FROM [SHOW TABLES] ORDER BY table_name ---- perm_table -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/unique_without_index_constraint b/pkg/ccl/backupccl/testdata/backup-restore/unique_without_index_constraint index 1cf1b335ceda..5dd911518387 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/unique_without_index_constraint +++ b/pkg/ccl/backupccl/testdata/backup-restore/unique_without_index_constraint @@ -1,4 +1,4 @@ -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -25,7 +25,7 @@ exec-sql BACKUP TABLE uwi INTO 'external://foo/table'; ---- -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions b/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions index 31cd137a3b74..c595d1a91eb4 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions +++ b/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions @@ -1,5 +1,5 @@ # Test backing up and restoring a database with user defined functions. -new-server name=s +new-cluster name=s ---- exec-sql @@ -144,10 +144,10 @@ DROP TYPE sc1.enum1 pq: cannot drop type "enum1" because other objects ([db1_new.sc1.f1]) still depend on it # Test backing up and restoring a full cluster with user defined function. -new-server name=s1 +new-cluster name=s1 ---- -exec-sql server=s1 +exec-sql cluster=s1 CREATE DATABASE db1; USE db1; CREATE SCHEMA sc1; @@ -219,11 +219,11 @@ SELECT sc1.f1('Good'::sc1.enum1) 1 # Start a new cluster with the same IO dir. -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- # Restore into the new cluster. -exec-sql server=s2 +exec-sql cluster=s2 RESTORE FROM LATEST IN 'nodelocal://0/test/' ---- diff --git a/pkg/ccl/backupccl/testdata/backup-restore/user-defined-types b/pkg/ccl/backupccl/testdata/backup-restore/user-defined-types index bc13e9c89c82..7bf5a44e14c9 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/user-defined-types +++ b/pkg/ccl/backupccl/testdata/backup-restore/user-defined-types @@ -1,6 +1,6 @@ # Test full cluster backup/restore here. -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -24,11 +24,11 @@ BACKUP INTO 'nodelocal://0/test/' ---- # Start a new cluster with the same IO dir. -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- # Restore into the new cluster. -exec-sql server=s2 +exec-sql cluster=s2 RESTORE FROM LATEST IN 'nodelocal://0/test/' ---- @@ -107,7 +107,7 @@ pq: cannot drop type "farewell" because other objects ([d2.public.t1 d2.public.t reset ---- -new-server name=s +new-cluster name=s ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/views b/pkg/ccl/backupccl/testdata/backup-restore/views index 89c8fce9da13..3e5d9528fee4 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/views +++ b/pkg/ccl/backupccl/testdata/backup-restore/views @@ -1,5 +1,5 @@ # Make sure that db names are rewritten in a view restored to a new db name. -new-server name=s +new-cluster name=s ---- exec-sql diff --git a/pkg/ccl/backupccl/testdata/backup-restore/virtual-columns b/pkg/ccl/backupccl/testdata/backup-restore/virtual-columns index 37cdb14e1b2b..4d7f1caff553 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/virtual-columns +++ b/pkg/ccl/backupccl/testdata/backup-restore/virtual-columns @@ -1,6 +1,6 @@ # Test full cluster backup/restore here. -new-server name=s1 +new-cluster name=s1 ---- exec-sql @@ -22,10 +22,10 @@ BACKUP INTO 'nodelocal://0/test/' ---- # Start a new cluster with the same IO dir. -new-server name=s2 share-io-dir=s1 +new-cluster name=s2 share-io-dir=s1 ---- -exec-sql server=s2 +exec-sql cluster=s2 RESTORE FROM LATEST IN 'nodelocal://0/test/' ----