From a2aad65a6c1c4def577fd62756568e47405cd56c Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Mon, 24 Jan 2022 12:34:03 -0500 Subject: [PATCH] backupccl,spanconfig,kvserver: ExportRequest noops on ranges excluded from backup This change is the first of two changes that gets us to the goal of backup ignoring certain table row data, and not holding up GC on these ranges. This change does a few things: - It sets up the transport of the exclude_data_from_backup bit set on a table descriptor, to the span configuration applied in KV. - It teaches ExportRequest on a range marked as excluded to return an empty ExportResponse. In this way, a backup processor will receive no row data to backup up for an ephemeral table. - A follow up change will also teach the SQLTranslator to not populate the protected timestamp field on the SpanConfig for such tables. This way, a long running backup will not hold up GC on such high-churn tables. With no protection on such ranges, it is possible that an ExportRequest targetting the range has a StartTime below the range's GCThreshold. To avoid the returned BatchTimestampBeforeGCError from failing the backup we decorate the the error with information about the range being excluded from backup and handle the error in the backup processor. Informs: #73536 Release note (sql change): BACKUP of a table marked with `exclude_data_from_backup` via `ALTER TABLE ... SET (exclude_data_from_backup = true)` will no longer backup that table's row data. The backup will continue to backup the table's descriptor and related metadata, and so on restore we will end up with an empty version of the backed up table. --- pkg/ccl/backupccl/BUILD.bazel | 8 +- pkg/ccl/backupccl/backup_cloud_test.go | 6 +- pkg/ccl/backupccl/backup_processor.go | 12 +- pkg/ccl/backupccl/backup_rand_test.go | 6 +- pkg/ccl/backupccl/backup_test.go | 704 +++++++++++------- pkg/ccl/backupccl/bench_test.go | 33 +- .../full_cluster_backup_restore_test.go | 108 +-- pkg/ccl/backupccl/helpers_test.go | 398 ---------- .../restore_mid_schema_change_test.go | 4 +- .../backupccl/restore_old_sequences_test.go | 2 +- .../backupccl/restore_old_versions_test.go | 48 +- pkg/ccl/backupccl/show_test.go | 42 +- pkg/ccl/backupccl/testutils.go | 471 +++++++++++- .../testdata/exclude_data_from_backup | 52 ++ .../testdata/tenant/exclude_data_from_backup | 52 ++ pkg/kv/kvserver/batcheval/cmd_export.go | 12 + pkg/kv/kvserver/batcheval/eval_context.go | 4 + pkg/kv/kvserver/replica.go | 17 +- pkg/kv/kvserver/replica_eval_context_span.go | 6 + pkg/roachpb/errors.proto | 4 + pkg/roachpb/span_config.proto | 8 + .../spanconfigsqltranslator/sqltranslator.go | 7 +- pkg/spanconfig/spanconfigtestutils/utils.go | 3 + 23 files changed, 1190 insertions(+), 817 deletions(-) delete mode 100644 pkg/ccl/backupccl/helpers_test.go create mode 100644 pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup create mode 100644 pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/exclude_data_from_backup diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 3221ca58d747..bd770e993303 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -75,6 +75,7 @@ go_library( "//pkg/sql/catalog/descidgen", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/desctestutils", "//pkg/sql/catalog/multiregion", "//pkg/sql/catalog/resolver", "//pkg/sql/catalog/schemadesc", @@ -103,6 +104,9 @@ go_library( "//pkg/sql/types", "//pkg/storage", "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", "//pkg/util", "//pkg/util/admission", "//pkg/util/contextutil", @@ -123,6 +127,8 @@ go_library( "//pkg/util/timeutil", "//pkg/util/tracing", "//pkg/util/uuid", + "//pkg/workload/bank", + "//pkg/workload/workloadsql", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", "@com_github_gogo_protobuf//jsonpb", @@ -148,7 +154,6 @@ go_test( "bench_test.go", "create_scheduled_backup_test.go", "full_cluster_backup_restore_test.go", - "helpers_test.go", "insert_missing_public_schema_namespace_entry_restore_test.go", "key_rewriter_test.go", "main_test.go", @@ -247,7 +252,6 @@ go_test( "//pkg/util/timeutil", "//pkg/util/uuid", "//pkg/workload/bank", - "//pkg/workload/workloadsql", "@com_github_aws_aws_sdk_go//aws/credentials", "@com_github_cockroachdb_cockroach_go_v2//crdb", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/ccl/backupccl/backup_cloud_test.go b/pkg/ccl/backupccl/backup_cloud_test.go index 7b5e8069143e..a02ba0b32509 100644 --- a/pkg/ccl/backupccl/backup_cloud_test.go +++ b/pkg/ccl/backupccl/backup_cloud_test.go @@ -62,7 +62,7 @@ func TestCloudBackupRestoreS3(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, 1, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts, InitManualReplication) defer cleanupFn() prefix := fmt.Sprintf("TestBackupRestoreS3-%d", timeutil.Now().UnixNano()) uri := url.URL{Scheme: "s3", Host: bucket, Path: prefix} @@ -89,7 +89,7 @@ func TestCloudBackupRestoreGoogleCloudStorage(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, 1, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts, InitManualReplication) defer cleanupFn() prefix := fmt.Sprintf("TestBackupRestoreGoogleCloudStorage-%d", timeutil.Now().UnixNano()) uri := url.URL{Scheme: "gs", Host: bucket, Path: prefix} @@ -120,7 +120,7 @@ func TestCloudBackupRestoreAzure(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, 1, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts, InitManualReplication) defer cleanupFn() prefix := fmt.Sprintf("TestBackupRestoreAzure-%d", timeutil.Now().UnixNano()) uri := url.URL{Scheme: "azure", Host: bucket, Path: prefix} diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index d5d3c824e6b0..f033d94583e0 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -451,6 +451,17 @@ func runBackupProcessor( if errors.HasType(exportRequestErr, (*contextutil.TimeoutError)(nil)) { return errors.Wrap(exportRequestErr, "export request timeout") } + // BatchTimestampBeforeGCError is returned if the ExportRequest + // attempts to read below the range's GC threshold. + if batchTimestampBeforeGCError, ok := pErr.GetDetail().(*roachpb.BatchTimestampBeforeGCError); ok { + // If the range we are exporting is marked to be excluded from + // backup, it is safe to ignore the error. It is likely that the + // table has been configured with a low GC TTL, and so the data + // the backup is targeting has already been gc'ed. + if batchTimestampBeforeGCError.DataExcludedFromBackup { + continue + } + } return errors.Wrapf(exportRequestErr, "exporting %s", span.span) } @@ -633,7 +644,6 @@ func makeSSTSink( // prior to the error. incrementSize := int64(32 << 20) maxSize := smallFileBuffer.Get(s.conf.settings) - log.Infof(ctx, "this is the max size we are using %d", (maxSize / (1024 * 1024))) for { if s.queueCap >= maxSize { break diff --git a/pkg/ccl/backupccl/backup_rand_test.go b/pkg/ccl/backupccl/backup_rand_test.go index 886b907381bc..89f2104702db 100644 --- a/pkg/ccl/backupccl/backup_rand_test.go +++ b/pkg/ccl/backupccl/backup_rand_test.go @@ -85,8 +85,8 @@ database_name = 'rand' AND schema_name = 'public'`) // Now that we've created our random tables, backup and restore the whole DB // and compare all table descriptors for equality. - dbBackup := LocalFoo + "wholedb" - tablesBackup := LocalFoo + "alltables" + dbBackup := localFoo + "wholedb" + tablesBackup := localFoo + "alltables" dbBackups := []string{dbBackup, tablesBackup} if err := verifyBackupRestoreStatementResult( @@ -136,7 +136,7 @@ database_name = 'rand' AND schema_name = 'public'`) for i, combo := range tableNameCombos { sqlDB.Exec(t, "DROP DATABASE IF EXISTS restoredb; CREATE DATABASE restoredb") - backupTarget := fmt.Sprintf("%s%d", LocalFoo, i) + backupTarget := fmt.Sprintf("%s%d", localFoo, i) if len(combo) == 0 { continue } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 3fede7e2af15..f2aa7b36b05c 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -54,6 +54,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -373,11 +374,11 @@ func TestBackupRestoreStatementResult(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() if err := verifyBackupRestoreStatementResult( - t, sqlDB, "BACKUP DATABASE data TO $1", LocalFoo, + t, sqlDB, "BACKUP DATABASE data TO $1", localFoo, ); err != nil { t.Fatal(err) } @@ -396,7 +397,7 @@ func TestBackupRestoreStatementResult(t *testing.T) { sqlDB.Exec(t, "CREATE DATABASE data2") if err := verifyBackupRestoreStatementResult( - t, sqlDB, "RESTORE data.* FROM $1 WITH OPTIONS (into_db='data2')", LocalFoo, + t, sqlDB, "RESTORE data.* FROM $1 WITH OPTIONS (into_db='data2')", localFoo, ); err != nil { t.Fatal(err) } @@ -408,7 +409,7 @@ func TestBackupRestoreSingleUserfile(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() backupAndRestore(ctx, t, tc, []string{"userfile:///a"}, []string{"userfile:///a"}, numAccounts) @@ -420,10 +421,10 @@ func TestBackupRestoreSingleNodeLocal(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - backupAndRestore(ctx, t, tc, []string{LocalFoo}, []string{LocalFoo}, numAccounts) + backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) } func TestBackupRestoreMultiNodeLocal(t *testing.T) { @@ -432,10 +433,10 @@ func TestBackupRestoreMultiNodeLocal(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() - backupAndRestore(ctx, t, tc, []string{LocalFoo}, []string{LocalFoo}, numAccounts) + backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) } func TestBackupRestoreMultiNodeRemote(t *testing.T) { @@ -444,12 +445,12 @@ func TestBackupRestoreMultiNodeRemote(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() // Backing up to node2's local file system remoteFoo := "nodelocal://2/foo" - backupAndRestore(ctx, t, tc, []string{remoteFoo}, []string{LocalFoo}, numAccounts) + backupAndRestore(ctx, t, tc, []string{remoteFoo}, []string{localFoo}, numAccounts) } func TestBackupRestorePartitioned(t *testing.T) { @@ -492,10 +493,10 @@ func TestBackupRestorePartitioned(t *testing.T) { _, sqlDB, dir, cleanupFn := backupRestoreTestSetupWithParams(t, 3 /* nodes */, numAccounts, InitManualReplication, args) defer cleanupFn() - // locationToDir converts backup URIs based on LocalFoo to the temporary + // locationToDir converts backup URIs based on localFoo to the temporary // file it represents on disk. locationToDir := func(location string) string { - return strings.Replace(location, LocalFoo, filepath.Join(dir, "foo"), 1) + return strings.Replace(location, localFoo, filepath.Join(dir, "foo"), 1) } hasSSTs := func(t *testing.T, location string) bool { @@ -586,9 +587,9 @@ func TestBackupRestorePartitioned(t *testing.T) { ensureLeaseholder(t, sqlDB) testSubDir := t.Name() locations := []string{ - LocalFoo + "/" + testSubDir + "/1", - LocalFoo + "/" + testSubDir + "/2", - LocalFoo + "/" + testSubDir + "/3", + localFoo + "/" + testSubDir + "/1", + localFoo + "/" + testSubDir + "/2", + localFoo + "/" + testSubDir + "/3", } backupURIs := []string{ // The first location will contain data from node 3 with config @@ -611,10 +612,10 @@ func TestBackupRestorePartitioned(t *testing.T) { ensureLeaseholder(t, sqlDB) testSubDir := t.Name() locations := []string{ - LocalFoo + "/" + testSubDir + "/1", - LocalFoo + "/" + testSubDir + "/2", - LocalFoo + "/" + testSubDir + "/3", - LocalFoo + "/" + testSubDir + "/4", + localFoo + "/" + testSubDir + "/1", + localFoo + "/" + testSubDir + "/2", + localFoo + "/" + testSubDir + "/3", + localFoo + "/" + testSubDir + "/4", } backupURIs := []string{ fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", locations[0], url.QueryEscape("default")), @@ -635,10 +636,10 @@ func TestBackupRestorePartitioned(t *testing.T) { ensureLeaseholder(t, sqlDB) testSubDir := t.Name() locations := []string{ - LocalFoo + "/" + testSubDir + "/1", - LocalFoo + "/" + testSubDir + "/2", - LocalFoo + "/" + testSubDir + "/3", - LocalFoo + "/" + testSubDir + "/4", + localFoo + "/" + testSubDir + "/1", + localFoo + "/" + testSubDir + "/2", + localFoo + "/" + testSubDir + "/3", + localFoo + "/" + testSubDir + "/4", } backupURIs := []string{ fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", locations[0], url.QueryEscape("default")), @@ -662,11 +663,11 @@ func TestBackupRestoreAppend(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, sqlDB, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + tc, sqlDB, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() // Ensure that each node has at least one leaseholder. (These splits were - // made in BackupRestoreTestSetup.) These are wrapped with SucceedsSoon() + // made in backupRestoreTestSetup.) These are wrapped with SucceedsSoon() // because EXPERIMENTAL_RELOCATE can fail if there are other replication // changes happening. for _, stmt := range []string{ @@ -679,7 +680,7 @@ func TestBackupRestoreAppend(t *testing.T) { return err }) } - const localFoo1, localFoo2, localFoo3 = LocalFoo + "/1", LocalFoo + "/2", LocalFoo + "/3" + const localFoo1, localFoo2, localFoo3 = localFoo + "/1", localFoo + "/2", localFoo + "/3" const userfileFoo1, userfileFoo2, userfileFoo3 = `userfile:///bar/1`, `userfile:///bar/2`, `userfile:///bar/3` makeBackups := func(b1, b2, b3 string) []interface{} { @@ -807,7 +808,7 @@ func TestBackupRestoreAppend(t *testing.T) { // restoring cluster needs to be empty, which means it can't contain any // userfile tables. - _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, MultiNode, tmpDir, InitManualReplication, base.TestClusterArgs{}) + _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, multiNode, tmpDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupEmptyCluster() sqlDBRestore.Exec(t, "RESTORE FROM $4 IN ($1, $2, $3) AS OF SYSTEM TIME "+ts2, append(collections, fullBackup2)...) } @@ -877,13 +878,13 @@ func TestBackupAndRestoreJobDescription(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, sqlDB, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() const c1, c2, c3 = `nodelocal://0/full/`, `nodelocal://1/full/`, `nodelocal://2/full/` const i1, i2, i3 = `nodelocal://0/inc/`, `nodelocal://1/inc/`, `nodelocal://2/inc/` - const localFoo1, localFoo2, localFoo3 = LocalFoo + "/1", LocalFoo + "/2", LocalFoo + "/3" + const localFoo1, localFoo2, localFoo3 = localFoo + "/1", localFoo + "/2", localFoo + "/3" backups := []interface{}{ fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", localFoo1, url.QueryEscape("default")), fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", localFoo2, url.QueryEscape("dc=dc1")), @@ -998,14 +999,14 @@ func TestBackupRestorePartitionedMergeDirectories(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() // TODO (lucy): This test writes a partitioned backup where all files are // written to the same directory, which is similar to the case where a backup // is created and then all files are consolidated into the same directory, but // we should still have a separate test where the files are actually moved. - const localFoo1 = LocalFoo + "/1" + const localFoo1 = localFoo + "/1" backupURIs := []string{ fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", localFoo1, url.QueryEscape("default")), fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", localFoo1, url.QueryEscape("dc=dc1")), @@ -1023,10 +1024,10 @@ func TestBackupRestoreEmpty(t *testing.T) { const numAccounts = 0 ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - backupAndRestore(ctx, t, tc, []string{LocalFoo}, []string{LocalFoo}, numAccounts) + backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) } // Regression test for #16008. In short, the way RESTORE constructed split keys @@ -1039,7 +1040,7 @@ func TestBackupRestoreNegativePrimaryKey(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() // Give half the accounts negative primary keys. @@ -1051,13 +1052,13 @@ func TestBackupRestoreNegativePrimaryKey(t *testing.T) { -numAccounts/2, numAccounts/backupRestoreDefaultRanges/2, ) - backupAndRestore(ctx, t, tc, []string{LocalFoo}, []string{LocalFoo}, numAccounts) + backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) sqlDB.Exec(t, `CREATE UNIQUE INDEX id2 ON data.bank (id)`) var unused string var exportedRows, exportedIndexEntries int - sqlDB.QueryRow(t, `BACKUP DATABASE data TO $1`, LocalFoo+"/alteredPK").Scan( + sqlDB.QueryRow(t, `BACKUP DATABASE data TO $1`, localFoo+"/alteredPK").Scan( &unused, &unused, &unused, &exportedRows, &exportedIndexEntries, &unused, ) if exportedRows != numAccounts { @@ -1257,7 +1258,7 @@ func TestBackupRestoreSystemTables(t *testing.T) { const numAccounts = 0 ctx := context.Background() - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) conn := sqlDB.DB.(*gosql.DB) defer cleanupFn() @@ -1291,8 +1292,8 @@ func TestBackupRestoreSystemTables(t *testing.T) { // Backup and restore the tables into a new database. sqlDB.Exec(t, `CREATE DATABASE system_new`) - sqlDB.Exec(t, fmt.Sprintf(`BACKUP %s TO '%s' AS OF SYSTEM TIME %s`, tableSpec, LocalFoo, backupAsOf)) - sqlDB.Exec(t, fmt.Sprintf(`RESTORE %s FROM '%s' WITH into_db='system_new'`, tableSpec, LocalFoo)) + sqlDB.Exec(t, fmt.Sprintf(`BACKUP %s TO '%s' AS OF SYSTEM TIME %s`, tableSpec, localFoo, backupAsOf)) + sqlDB.Exec(t, fmt.Sprintf(`RESTORE %s FROM '%s' WITH into_db='system_new'`, tableSpec, localFoo)) // Verify the fingerprints match. for _, table := range tables { @@ -1307,7 +1308,7 @@ func TestBackupRestoreSystemTables(t *testing.T) { // directly over the existing system tables. sqlDB.ExpectErr( t, `relation ".+" already exists`, - fmt.Sprintf(`RESTORE %s FROM '%s'`, tableSpec, LocalFoo), + fmt.Sprintf(`RESTORE %s FROM '%s'`, tableSpec, localFoo), ) } @@ -1316,14 +1317,14 @@ func TestBackupRestoreSystemJobs(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 0 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) conn := sqlDB.DB.(*gosql.DB) defer cleanupFn() - sanitizedIncDir := LocalFoo + "/inc?AWS_SESSION_TOKEN=" + sanitizedIncDir := localFoo + "/inc?AWS_SESSION_TOKEN=" incDir := sanitizedIncDir + "secretCredentialsHere" - sanitizedFullDir := LocalFoo + "/full?AWS_SESSION_TOKEN=" + sanitizedFullDir := localFoo + "/full?AWS_SESSION_TOKEN=" fullDir := sanitizedFullDir + "moarSecretsHere" backupDatabaseID := sqlutils.QueryDatabaseID(t, conn, "data") @@ -1434,10 +1435,10 @@ func TestEncryptedBackupRestoreSystemJobs(t *testing.T) { sanitizedEncryptionOption1 = "encryption_passphrase = '*****'" sanitizedEncryptionOption2 = "encryption_passphrase = 'redacted'" } - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, 3, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, 3, InitManualReplication) conn := sqlDB.DB.(*gosql.DB) defer cleanupFn() - backupLoc1 := LocalFoo + "/x" + backupLoc1 := localFoo + "/x" sqlDB.Exec(t, `CREATE DATABASE restoredb`) backupDatabaseID := sqlutils.QueryDatabaseID(t, conn, "data") @@ -1537,7 +1538,7 @@ func checkInProgressBackupRestore( const numAccounts = 100 ctx := context.Background() - _, sqlDB, dir, cleanup := backupRestoreTestSetupWithParams(t, MultiNode, numAccounts, + _, sqlDB, dir, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, InitManualReplication, params) conn := sqlDB.DB.(*gosql.DB) defer cleanup() @@ -1549,7 +1550,7 @@ func checkInProgressBackupRestore( sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.merge_file_buffer_size = '1'`) // Ensure that each node has at least one leaseholder. (These splits were - // made in BackupRestoreTestSetup.) These are wrapped with SucceedsSoon() + // made in backupRestoreTestSetup.) These are wrapped with SucceedsSoon() // because EXPERIMENTAL_RELOCATE can fail if there are other replication // changes happening. for _, stmt := range []string{ @@ -1617,7 +1618,7 @@ WHERE allowResponse = make(chan struct{}, totalExpectedResponses) go func() { - _, err := conn.Exec(query, LocalFoo) + _, err := conn.Exec(query, localFoo) jobDone <- err }() @@ -1800,7 +1801,7 @@ func TestBackupRestoreResume(t *testing.T) { }} const numAccounts = 1000 - tc, outerDB, dir, cleanupFn := backupRestoreTestSetupWithParams(t, MultiNode, numAccounts, InitManualReplication, params) + tc, outerDB, dir, cleanupFn := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, InitManualReplication, params) defer cleanupFn() backupTableDesc := desctestutils.TestingGetPublicTableDescriptor(tc.Servers[0].DB(), keys.SystemSQLCodec, "data", "bank") @@ -1979,7 +1980,7 @@ func TestBackupRestoreControlJob(t *testing.T) { config.TestingSetZoneConfig(last+1, zoneConfig) } const numAccounts = 1000 - _, outerDB, _, cleanup := backupRestoreTestSetupWithParams(t, MultiNode, numAccounts, init, params) + _, outerDB, _, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, init, params) defer cleanup() sqlDB := sqlutils.MakeSQLRunner(outerDB.DB) @@ -2114,7 +2115,7 @@ func TestRestoreFailCleansUpTypeBackReferences(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, dir, cleanup := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, dir, cleanup := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanup() dir = dir + "/foo" @@ -2128,7 +2129,7 @@ INSERT INTO d.tb VALUES ('hello'), ('hello'); `) // Backup d.tb. - sqlDB.Exec(t, `BACKUP TABLE d.tb TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TABLE d.tb TO $1`, localFoo) // Drop d.tb so that it can be restored. sqlDB.Exec(t, `DROP TABLE d.tb`) @@ -2147,7 +2148,7 @@ INSERT INTO d.tb VALUES ('hello'), ('hello'); } // We should get an error when restoring the table. - sqlDB.ExpectErr(t, "sst: no such file", `RESTORE d.tb FROM $1`, LocalFoo) + sqlDB.ExpectErr(t, "sst: no such file", `RESTORE d.tb FROM $1`, localFoo) // The failed restore should clean up type back references so that we are able // to drop d.ty. @@ -2187,7 +2188,7 @@ func TestRestoreFailCleanup(t *testing.T) { // Do the same with a user defined schema. sqlDB.Exec(t, `USE data; CREATE SCHEMA myschema`) - sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo) // Bugger the backup by removing the SST files. if err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error { if err != nil { @@ -2202,7 +2203,7 @@ func TestRestoreFailCleanup(t *testing.T) { } sqlDB.ExpectErr( t, "sst: no such file", - `RESTORE data.* FROM $1 WITH OPTIONS (into_db='restore')`, LocalFoo, + `RESTORE data.* FROM $1 WITH OPTIONS (into_db='restore')`, localFoo, ) // Verify the failed RESTORE added some DROP tables. sqlDB.CheckQueryResults(t, @@ -2244,7 +2245,7 @@ func TestRestoreFailDatabaseCleanup(t *testing.T) { // Do the same with a user defined schema. sqlDB.Exec(t, `USE data; CREATE SCHEMA myschema`) - sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo) // Bugger the backup by removing the SST files. if err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error { if err != nil { @@ -2260,7 +2261,7 @@ func TestRestoreFailDatabaseCleanup(t *testing.T) { sqlDB.Exec(t, `DROP DATABASE data`) sqlDB.ExpectErr( t, "sst: no such file", - `RESTORE DATABASE data FROM $1`, LocalFoo, + `RESTORE DATABASE data FROM $1`, localFoo, ) sqlDB.ExpectErr( t, `database "data" does not exist`, @@ -2272,7 +2273,7 @@ func TestRestoreFailCleansUpTempSystemDatabase(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, dir, cleanup := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, dir, cleanup := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanup() // Create a database with a type and table. @@ -2284,7 +2285,7 @@ INSERT INTO d.tb VALUES ('hello'), ('hello'); `) // Cluster BACKUP. - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) // Bugger the backup by removing the SST files. if err := filepath.Walk(dir+"/foo", func(path string, info os.FileInfo, err error) error { @@ -2308,7 +2309,7 @@ INSERT INTO d.tb VALUES ('hello'), ('hello'); ).Scan(&defaultDBID) // We should get an error when restoring the table. - sqlDBRestore.ExpectErr(t, "sst: no such file", `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.ExpectErr(t, "sst: no such file", `RESTORE FROM $1`, localFoo) // Make sure the temp system db is not present. row := sqlDBRestore.QueryStr(t, fmt.Sprintf(`SELECT * FROM [SHOW DATABASES] WHERE database_name = '%s'`, restoreTempSystemDB)) @@ -2331,7 +2332,7 @@ func TestBackupRestoreUserDefinedSchemas(t *testing.T) { // history at certain timestamps, then restores to each of the timestamps to // ensure that the types restored are correct. t.Run("revision-history", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() var ts1, ts2, ts3, ts4, ts5, ts6 string @@ -2420,7 +2421,7 @@ CREATE TABLE sc.t1 (a FLOAT); // Tests full cluster backup/restore with user defined schemas. t.Run("full-cluster", func(t *testing.T) { - _, sqlDB, dataDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, dataDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` CREATE DATABASE d; @@ -2454,7 +2455,7 @@ INSERT INTO sc.tb2 VALUES ('hello'); // Tests restoring databases with user defined schemas. t.Run("database", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` @@ -2488,7 +2489,7 @@ INSERT INTO sc.tb2 VALUES ('hello'); // Tests backing up and restoring all tables in requested user defined // schemas. t.Run("all-tables-in-requested-schema", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` @@ -2554,7 +2555,7 @@ table_name from [SHOW TABLES FROM restore] ORDER BY schema_name, table_name`, tc // Test restoring tables with user defined schemas when restore schemas are // not being remapped. t.Run("no-remap", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` @@ -2611,7 +2612,7 @@ INSERT INTO sc.tb2 VALUES (1); // Test restoring tables with user defined schemas when restore schemas are // not being remapped. Like no-remap but with more databases and schemas. t.Run("multi-schemas", func(t *testing.T) { - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() kvDB := tc.Server(0).DB() @@ -2670,7 +2671,7 @@ INSERT INTO sc4.tb VALUES (4); }) // Test when we remap schemas to existing schemas in the cluster. t.Run("remap", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` @@ -2705,7 +2706,7 @@ func TestBackupRestoreUserDefinedTypes(t *testing.T) { // ts4: no farewell type exists // ts5: farewell type exists as (third) t.Run("revision-history", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() var ts1, ts2, ts3, ts4, ts5 string @@ -2832,7 +2833,7 @@ RESTORE DATABASE d FROM 'nodelocal://0/rev-history-backup' // Test backup/restore of a single table. t.Run("table", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` CREATE DATABASE d; @@ -2893,7 +2894,7 @@ INSERT INTO d.t3 VALUES ('hi'); // Test cases where we attempt to remap types in the backup to types that // already exist in the cluster. t.Run("backup-remap", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` CREATE DATABASE d; @@ -3006,7 +3007,7 @@ INSERT INTO d.t2 VALUES (ARRAY['hello']); // Test cases where we attempt to remap types in the backup to types that // already exist in the cluster with user defined schema. t.Run("backup-remap-uds", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` CREATE DATABASE d; @@ -3020,9 +3021,9 @@ INSERT INTO d.s.t2 VALUES (ARRAY['hello']); `) { // Backup and restore t. - sqlDB.Exec(t, `BACKUP TABLE d.s.t TO $1`, LocalFoo+"/1") + sqlDB.Exec(t, `BACKUP TABLE d.s.t TO $1`, localFoo+"/1") sqlDB.Exec(t, `DROP TABLE d.s.t`) - sqlDB.Exec(t, `RESTORE TABLE d.s.t FROM $1`, LocalFoo+"/1") + sqlDB.Exec(t, `RESTORE TABLE d.s.t FROM $1`, localFoo+"/1") // Check that the table data is restored correctly and the types aren't touched. sqlDB.CheckQueryResults(t, `SELECT 'hello'::d.s.greeting, ARRAY['hello']::d.s.greeting[]`, [][]string{{"hello", "{hello}"}}) @@ -3035,9 +3036,9 @@ INSERT INTO d.s.t2 VALUES (ARRAY['hello']); { // Test that backing up and restoring a table with just the array type // will remap types appropriately. - sqlDB.Exec(t, `BACKUP TABLE d.s.t2 TO $1`, LocalFoo+"/2") + sqlDB.Exec(t, `BACKUP TABLE d.s.t2 TO $1`, localFoo+"/2") sqlDB.Exec(t, `DROP TABLE d.s.t2`) - sqlDB.Exec(t, `RESTORE TABLE d.s.t2 FROM $1`, LocalFoo+"/2") + sqlDB.Exec(t, `RESTORE TABLE d.s.t2 FROM $1`, localFoo+"/2") sqlDB.CheckQueryResults(t, `SELECT 'hello'::d.s.greeting, ARRAY['hello']::d.s.greeting[]`, [][]string{{"hello", "{hello}"}}) sqlDB.CheckQueryResults(t, `SELECT * FROM d.s.t2 ORDER BY x`, [][]string{{"{hello}"}}) } @@ -3049,14 +3050,14 @@ INSERT INTO d.s.t2 VALUES (ARRAY['hello']); sqlDB.Exec(t, `CREATE TYPE d2.s.greeting AS ENUM ('hello', 'howdy', 'hi')`) // Now restore t into this database. It should remap d.greeting to d2.greeting. - sqlDB.Exec(t, `RESTORE TABLE d.s.t FROM $1 WITH into_db = 'd2'`, LocalFoo+"/1") + sqlDB.Exec(t, `RESTORE TABLE d.s.t FROM $1 WITH into_db = 'd2'`, localFoo+"/1") // d.t should be added as a back reference to greeting. sqlDB.ExpectErr(t, `pq: cannot drop type "greeting" because other objects \(.*\) still depend on it`, `DROP TYPE d2.s.greeting`) } }) t.Run("incremental", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` CREATE DATABASE d; @@ -3265,7 +3266,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { const createStore = "CREATE DATABASE store" const createStoreStats = "CREATE DATABASE storestats" - _, origDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: dir} @@ -3328,7 +3329,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { } } } - _ = origDB.Exec(t, `BACKUP DATABASE store, storestats TO $1`, LocalFoo) + _ = origDB.Exec(t, `BACKUP DATABASE store, storestats TO $1`, localFoo) } origCustomers := origDB.QueryStr(t, `SHOW CONSTRAINTS FROM store.customers`) @@ -3344,7 +3345,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { db := sqlutils.MakeSQLRunner(tc.Conns[0]) db.Exec(t, createStore) - db.Exec(t, `RESTORE store.* FROM $1`, LocalFoo) + db.Exec(t, `RESTORE store.* FROM $1`, localFoo) // Restore's Validate checks all the tables point to each other correctly. db.CheckQueryResults(t, `SHOW CONSTRAINTS FROM store.customers`, origCustomers) @@ -3381,7 +3382,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { defer tc.Stopper().Stop(context.Background()) db := sqlutils.MakeSQLRunner(tc.Conns[0]) db.Exec(t, createStore) - db.Exec(t, `RESTORE store.customers, store.orders FROM $1`, LocalFoo) + db.Exec(t, `RESTORE store.customers, store.orders FROM $1`, localFoo) // Restore's Validate checks all the tables point to each other correctly. // FK validation on customers from orders is preserved. @@ -3403,10 +3404,10 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { // FK validation of self-FK is preserved. db.ExpectErr( t, "cannot restore table \"orders\" without referenced table .* \\(or \"skip_missing_foreign_keys\" option\\)", - `RESTORE store.orders FROM $1`, LocalFoo, + `RESTORE store.orders FROM $1`, localFoo, ) - db.Exec(t, `RESTORE store.orders FROM $1 WITH OPTIONS (skip_missing_foreign_keys)`, LocalFoo) + db.Exec(t, `RESTORE store.orders FROM $1 WITH OPTIONS (skip_missing_foreign_keys)`, localFoo) // Restore's Validate checks all the tables point to each other correctly. // FK validation is gone. @@ -3419,7 +3420,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { defer tc.Stopper().Stop(context.Background()) db := sqlutils.MakeSQLRunner(tc.Conns[0]) db.Exec(t, createStore) - db.Exec(t, `RESTORE store.receipts FROM $1 WITH OPTIONS (skip_missing_foreign_keys)`, LocalFoo) + db.Exec(t, `RESTORE store.receipts FROM $1 WITH OPTIONS (skip_missing_foreign_keys)`, localFoo) // Restore's Validate checks all the tables point to each other correctly. // FK validation of orders and customer is gone. @@ -3437,7 +3438,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { defer tc.Stopper().Stop(context.Background()) db := sqlutils.MakeSQLRunner(tc.Conns[0]) db.Exec(t, createStore) - db.Exec(t, `RESTORE store.receipts, store.customers FROM $1 WITH OPTIONS (skip_missing_foreign_keys)`, LocalFoo) + db.Exec(t, `RESTORE store.receipts, store.customers FROM $1 WITH OPTIONS (skip_missing_foreign_keys)`, localFoo) // Restore's Validate checks all the tables point to each other correctly. // FK validation of orders is gone. @@ -3469,9 +3470,9 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { db.Exec(t, createStore) db.ExpectErr( t, `cannot restore view "early_customers" without restoring referenced table`, - `RESTORE store.early_customers FROM $1`, LocalFoo, + `RESTORE store.early_customers FROM $1`, localFoo, ) - db.Exec(t, `RESTORE store.early_customers, store.customers, store.orders FROM $1`, LocalFoo) + db.Exec(t, `RESTORE store.early_customers, store.customers, store.orders FROM $1`, localFoo) db.CheckQueryResults(t, `SELECT * FROM store.early_customers`, origEarlyCustomers) // nothing depends on orders so it can be dropped. @@ -3500,7 +3501,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { db.ExpectErr( t, `cannot restore view "ordercounts" without restoring referenced table`, - `RESTORE DATABASE storestats FROM $1`, LocalFoo, + `RESTORE DATABASE storestats FROM $1`, localFoo, ) db.Exec(t, createStore) @@ -3508,10 +3509,10 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { db.ExpectErr( t, `cannot restore view "ordercounts" without restoring referenced table`, - `RESTORE storestats.ordercounts, store.customers FROM $1`, LocalFoo, + `RESTORE storestats.ordercounts, store.customers FROM $1`, localFoo, ) - db.Exec(t, `RESTORE store.customers, storestats.ordercounts, store.orders FROM $1`, LocalFoo) + db.Exec(t, `RESTORE store.customers, storestats.ordercounts, store.orders FROM $1`, localFoo) // we want to observe just the view-related errors, not fk errors below. db.Exec(t, `ALTER TABLE store.orders DROP CONSTRAINT orders_customerid_fkey`) @@ -3535,7 +3536,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { db.CheckQueryResults(t, `SELECT * FROM storestats.ordercounts ORDER BY id`, origOrderCounts) db.Exec(t, `CREATE DATABASE otherstore`) - db.Exec(t, `RESTORE store.* FROM $1 WITH into_db = 'otherstore'`, LocalFoo) + db.Exec(t, `RESTORE store.* FROM $1 WITH into_db = 'otherstore'`, localFoo) // we want to observe just the view-related errors, not fk errors below. db.Exec(t, `ALTER TABLE otherstore.orders DROP CONSTRAINT orders_customerid_fkey`) db.Exec(t, `DROP TABLE otherstore.receipts`) @@ -3560,20 +3561,20 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { // Test cases where, after filtering out views that can't be restored, there are no other tables to restore - db.Exec(t, `RESTORE DATABASE storestats from $1 WITH OPTIONS (skip_missing_views)`, LocalFoo) - db.Exec(t, `RESTORE storestats.ordercounts from $1 WITH OPTIONS (skip_missing_views)`, LocalFoo) + db.Exec(t, `RESTORE DATABASE storestats from $1 WITH OPTIONS (skip_missing_views)`, localFoo) + db.Exec(t, `RESTORE storestats.ordercounts from $1 WITH OPTIONS (skip_missing_views)`, localFoo) // Ensure that the views were not restored since they are missing the tables they reference. db.CheckQueryResults(t, `USE storestats; SHOW TABLES;`, [][]string{}) // Need to specify into_db otherwise the restore gives error: // a database named "store" needs to exist to restore schema "public". - db.Exec(t, `RESTORE store.early_customers, store.referencing_early_customers from $1 WITH OPTIONS (skip_missing_views, into_db='storestats')`, LocalFoo) + db.Exec(t, `RESTORE store.early_customers, store.referencing_early_customers from $1 WITH OPTIONS (skip_missing_views, into_db='storestats')`, localFoo) // Ensure that the views were not restored since they are missing the tables they reference. db.CheckQueryResults(t, `SHOW TABLES;`, [][]string{}) // Test that views with valid dependencies are restored - db.Exec(t, `RESTORE DATABASE store from $1 WITH OPTIONS (skip_missing_views)`, LocalFoo) + db.Exec(t, `RESTORE DATABASE store from $1 WITH OPTIONS (skip_missing_views)`, localFoo) db.CheckQueryResults(t, `SELECT * FROM store.early_customers`, origEarlyCustomers) db.CheckQueryResults(t, `SELECT * FROM store.referencing_early_customers`, origEarlyCustomers) // TODO(lucy, jordan): DROP DATABASE CASCADE doesn't work in the mixed 19.1/ @@ -3588,7 +3589,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) { // db.Exec(t, createStore) // storestats.ordercounts depends also on store.orders, so it can't be restored db.Exec(t, `CREATE DATABASE store2`) - db.Exec(t, `RESTORE storestats.ordercounts, store.customers from $1 WITH OPTIONS (skip_missing_views, into_db='store2')`, LocalFoo) + db.Exec(t, `RESTORE storestats.ordercounts, store.customers from $1 WITH OPTIONS (skip_missing_views, into_db='store2')`, localFoo) db.CheckQueryResults(t, `SHOW CONSTRAINTS FROM store2.customers`, origCustomers) db.ExpectErr(t, `relation "storestats.ordercounts" does not exist`, `SELECT * FROM storestats.ordercounts`) }) @@ -3622,7 +3623,7 @@ func TestBackupRestoreIncremental(t *testing.T) { const numBackups = 4 windowSize := int(numAccounts / 3) - tc, sqlDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: dir} rng, _ := randutil.NewTestRand() @@ -3714,7 +3715,7 @@ func TestBackupRestorePartitionedIncremental(t *testing.T) { const numBackups = 4 windowSize := int(numAccounts / 3) - _, sqlDB, dir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, 0, InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, multiNode, 0, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: dir} rng, _ := randutil.NewTestRand() @@ -3842,10 +3843,10 @@ func TestBackupRestoreWithConcurrentWrites(t *testing.T) { defer log.Scope(t).Close(t) const rows = 10 - const numBackgroundTasks = MultiNode + const numBackgroundTasks = multiNode ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, rows, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, rows, InitManualReplication) defer cleanupFn() bgActivity := make(chan struct{}) @@ -3878,12 +3879,12 @@ func TestBackupRestoreWithConcurrentWrites(t *testing.T) { <-bgActivity // Backup DB while concurrent writes continue. - sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo) // Drop the table and restore from backup and check our invariant. atomic.StoreInt32(&allowErrors, 1) sqlDB.Exec(t, `DROP TABLE data.bank`) - sqlDB.Exec(t, `RESTORE data.* FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.* FROM $1`, localFoo) atomic.StoreInt32(&allowErrors, 0) bad := sqlDB.QueryStr(t, `SELECT id, balance, payload FROM data.bank WHERE id != balance`) @@ -3899,7 +3900,7 @@ func TestConcurrentBackupRestores(t *testing.T) { const numAccounts = 10 const concurrency, numIterations = 2, 3 ctx := context.Background() - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() g, gCtx := errgroup.WithContext(ctx) @@ -3947,7 +3948,7 @@ func TestBackupTenantsWithRevisionHistory(t *testing.T) { const numAccounts = 1 ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() _, err := tc.Servers[0].StartTenant(ctx, base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10)}) @@ -3969,7 +3970,7 @@ func TestBackupAsOfSystemTime(t *testing.T) { const numAccounts = 1000 ctx := context.Background() - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() var beforeTs, equalTs string @@ -3993,9 +3994,9 @@ func TestBackupAsOfSystemTime(t *testing.T) { t.Fatalf("expected %d rows but found %d", expected, rowCount) } - beforeDir := LocalFoo + `/beforeTs` + beforeDir := localFoo + `/beforeTs` sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO '%s' AS OF SYSTEM TIME %s`, beforeDir, beforeTs)) - equalDir := LocalFoo + `/equalTs` + equalDir := localFoo + `/equalTs` sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO '%s' AS OF SYSTEM TIME %s`, equalDir, equalTs)) sqlDB.Exec(t, `DROP TABLE data.bank`) @@ -4019,7 +4020,7 @@ func TestRestoreAsOfSystemTime(t *testing.T) { const numAccounts = 10 ctx := context.Background() - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() const dir = "nodelocal://0/" @@ -4310,7 +4311,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { const numAccounts = 10 ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() const dir = "nodelocal://0/" preGC := tree.TimestampToDecimalDatum(tc.Server(0).Clock().Now()).String() @@ -4345,7 +4346,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { t.Run("restore-pre-gc-aost", func(t *testing.T) { backupPath := dir + "/tbl-before-gc" - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, "CREATE DATABASE db") @@ -4373,15 +4374,15 @@ func TestAsOfSystemTimeOnRestoredData(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP data.* To $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.* To $1`, localFoo) sqlDB.Exec(t, `DROP TABLE data.bank`) var beforeTs string sqlDB.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&beforeTs) - sqlDB.Exec(t, `RESTORE data.* FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.* FROM $1`, localFoo) var afterTs string sqlDB.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&afterTs) @@ -4404,11 +4405,11 @@ func TestBackupRestoreChecksum(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1000 - _, sqlDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() dir = filepath.Join(dir, "foo") - sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo) var backupManifest BackupManifest { @@ -4443,7 +4444,7 @@ func TestBackupRestoreChecksum(t *testing.T) { } sqlDB.Exec(t, `DROP TABLE data.bank`) - sqlDB.ExpectErr(t, "checksum mismatch", `RESTORE data.* FROM $1`, LocalFoo) + sqlDB.ExpectErr(t, "checksum mismatch", `RESTORE data.* FROM $1`, localFoo) } func TestTimestampMismatch(t *testing.T) { @@ -4451,16 +4452,16 @@ func TestTimestampMismatch(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE TABLE data.t2 (a INT PRIMARY KEY)`) sqlDB.Exec(t, `INSERT INTO data.t2 VALUES (1)`) - fullBackup := LocalFoo + "/0" - incrementalT1FromFull := LocalFoo + "/1" - incrementalT2FromT1 := LocalFoo + "/2" - incrementalT3FromT1OneTable := LocalFoo + "/3" + fullBackup := localFoo + "/0" + incrementalT1FromFull := localFoo + "/1" + incrementalT2FromT1 := localFoo + "/2" + incrementalT3FromT1OneTable := localFoo + "/3" sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, fullBackup) @@ -4476,28 +4477,28 @@ func TestTimestampMismatch(t *testing.T) { sqlDB.ExpectErr( t, "backups listed out of order", `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2`, - LocalFoo, incrementalT1FromFull, + localFoo, incrementalT1FromFull, ) // Missing an intermediate incremental backup. sqlDB.ExpectErr( t, "backups listed out of order", `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2, $3`, - LocalFoo, fullBackup, incrementalT2FromT1, + localFoo, fullBackup, incrementalT2FromT1, ) // Backups specified out of order. sqlDB.ExpectErr( t, "out of order", `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2, $3`, - LocalFoo, incrementalT1FromFull, fullBackup, + localFoo, incrementalT1FromFull, fullBackup, ) // Missing data for one table in the most recent backup. sqlDB.ExpectErr( t, "previous backup does not contain table", `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2, $3`, - LocalFoo, fullBackup, incrementalT3FromT1OneTable, + localFoo, fullBackup, incrementalT3FromT1OneTable, ) }) @@ -4678,19 +4679,19 @@ func TestEncryptedBackup(t *testing.T) { incorrectEncryptionOption = "encryption_passphrase = 'wrongpassphrase'" } ctx := context.Background() - _, sqlDB, rawDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, 3, InitManualReplication) + _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, multiNode, 3, InitManualReplication) defer cleanupFn() setupBackupEncryptedTest(ctx, t, sqlDB) // Full cluster-backup to capture all possible metadata. - backupLoc1 := LocalFoo + "/x?COCKROACH_LOCALITY=default" - backupLoc2 := LocalFoo + "/x2?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") - backupLoc1inc := LocalFoo + "/inc1/x?COCKROACH_LOCALITY=default" - backupLoc2inc := LocalFoo + "/inc1/x2?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") + backupLoc1 := localFoo + "/x?COCKROACH_LOCALITY=default" + backupLoc2 := localFoo + "/x2?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") + backupLoc1inc := localFoo + "/inc1/x?COCKROACH_LOCALITY=default" + backupLoc2inc := localFoo + "/inc1/x2?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") - plainBackupLoc1 := LocalFoo + "/cleartext?COCKROACH_LOCALITY=default" - plainBackupLoc2 := LocalFoo + "/cleartext?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") + plainBackupLoc1 := localFoo + "/cleartext?COCKROACH_LOCALITY=default" + plainBackupLoc2 := localFoo + "/cleartext?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") sqlDB.Exec(t, `BACKUP TO ($1, $2)`, plainBackupLoc1, plainBackupLoc2) @@ -4770,14 +4771,14 @@ func TestRegionalKMSEncryptedBackup(t *testing.T) { t.Run("multi-region-kms", func(t *testing.T) { ctx := context.Background() - _, sqlDB, rawDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, 3, InitManualReplication) + _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, multiNode, 3, InitManualReplication) defer cleanupFn() setupBackupEncryptedTest(ctx, t, sqlDB) // Full cluster-backup to capture all possible metadata. - backupLoc1 := LocalFoo + "/x?COCKROACH_LOCALITY=default" - backupLoc2 := LocalFoo + "/x2?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") + backupLoc1 := localFoo + "/x?COCKROACH_LOCALITY=default" + backupLoc2 := localFoo + "/x2?COCKROACH_LOCALITY=" + url.QueryEscape("dc=dc1") sqlDB.Exec(t, fmt.Sprintf(`BACKUP TO ($1, $2) WITH %s`, concatMultiRegionKMSURIs(multiRegionKMSURIs)), backupLoc1, @@ -5003,7 +5004,7 @@ func TestRestoredPrivileges(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: dir} @@ -5020,7 +5021,7 @@ func TestRestoredPrivileges(t *testing.T) { withGrants := sqlDB.QueryStr(t, `SHOW GRANTS ON data.bank`) - sqlDB.Exec(t, `BACKUP DATABASE data, data2 TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data, data2 TO $1`, localFoo) sqlDB.Exec(t, `DROP TABLE data.bank`) t.Run("into fresh db", func(t *testing.T) { @@ -5028,7 +5029,7 @@ func TestRestoredPrivileges(t *testing.T) { defer tc.Stopper().Stop(context.Background()) sqlDBRestore := sqlutils.MakeSQLRunner(tc.Conns[0]) sqlDBRestore.Exec(t, `CREATE DATABASE data`) - sqlDBRestore.Exec(t, `RESTORE data.bank FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE data.bank FROM $1`, localFoo) sqlDBRestore.CheckQueryResults(t, `SHOW GRANTS ON data.bank`, rootOnly) }) @@ -5040,7 +5041,7 @@ func TestRestoredPrivileges(t *testing.T) { sqlDBRestore.Exec(t, `CREATE USER someone`) sqlDBRestore.Exec(t, `USE data`) sqlDBRestore.Exec(t, `ALTER DEFAULT PRIVILEGES GRANT SELECT, INSERT, UPDATE, DELETE ON TABLES TO someone`) - sqlDBRestore.Exec(t, `RESTORE data.bank FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE data.bank FROM $1`, localFoo) sqlDBRestore.CheckQueryResults(t, `SHOW GRANTS ON data.bank`, withGrants) }) @@ -5049,7 +5050,7 @@ func TestRestoredPrivileges(t *testing.T) { defer tc.Stopper().Stop(context.Background()) sqlDBRestore := sqlutils.MakeSQLRunner(tc.Conns[0]) sqlDBRestore.Exec(t, `CREATE USER someone`) - sqlDBRestore.Exec(t, `RESTORE DATABASE data2 FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE DATABASE data2 FROM $1`, localFoo) sqlDBRestore.CheckQueryResults(t, `SHOW GRANTS ON DATABASE data2`, data2Grants) }) } @@ -5059,12 +5060,12 @@ func TestRestoreInto(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo) - restoreStmt := fmt.Sprintf(`RESTORE data.bank FROM '%s' WITH into_db = 'data 2'`, LocalFoo) + restoreStmt := fmt.Sprintf(`RESTORE data.bank FROM '%s' WITH into_db = 'data 2'`, localFoo) sqlDB.ExpectErr(t, "a database named \"data 2\" needs to exist", restoreStmt) @@ -5080,7 +5081,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - tc, origDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, origDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: tc.Servers[0].ClusterSettings().ExternalIODir} @@ -5099,7 +5100,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { d4foobar := "nodelocal://0/d4foobar" d4star := "nodelocal://0/d4star" - origDB.Exec(t, `BACKUP DATABASE data, d2, d3, d4 TO $1`, LocalFoo) + origDB.Exec(t, `BACKUP DATABASE data, d2, d3, d4 TO $1`, localFoo) origDB.Exec(t, `BACKUP d4.foo TO $1`, d4foo) origDB.Exec(t, `BACKUP d4.foo, d4.bar TO $1`, d4foobar) origDB.Exec(t, `BACKUP d4.* TO $1`, d4star) @@ -5144,7 +5145,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { tcRestore := testcluster.StartTestCluster(t, singleNode, base.TestClusterArgs{ServerArgs: args}) defer tcRestore.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(tcRestore.Conns[0]) - sqlDB.Exec(t, `RESTORE DATABASE data, d2, d3 FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE DATABASE data, d2, d3 FROM $1`, localFoo) }) t.Run("db-exists", func(t *testing.T) { @@ -5153,7 +5154,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(tcRestore.Conns[0]) sqlDB.Exec(t, `CREATE DATABASE data`) - sqlDB.ExpectErr(t, "already exists", `RESTORE DATABASE data FROM $1`, LocalFoo) + sqlDB.ExpectErr(t, "already exists", `RESTORE DATABASE data FROM $1`, localFoo) }) t.Run("tables", func(t *testing.T) { @@ -5162,7 +5163,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(tcRestore.Conns[0]) sqlDB.Exec(t, `CREATE DATABASE data`) - sqlDB.Exec(t, `RESTORE data.* FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.* FROM $1`, localFoo) }) t.Run("tables-needs-db", func(t *testing.T) { @@ -5170,7 +5171,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { defer tcRestore.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(tcRestore.Conns[0]) - sqlDB.ExpectErr(t, "needs to exist", `RESTORE data.*, d4.* FROM $1`, LocalFoo) + sqlDB.ExpectErr(t, "needs to exist", `RESTORE data.*, d4.* FROM $1`, localFoo) }) t.Run("into_db", func(t *testing.T) { @@ -5180,7 +5181,7 @@ func TestRestoreDatabaseVersusTable(t *testing.T) { sqlDB.ExpectErr( t, `cannot use "into_db"`, - `RESTORE DATABASE data FROM $1 WITH into_db = 'other'`, LocalFoo, + `RESTORE DATABASE data FROM $1 WITH into_db = 'other'`, localFoo, ) }) } @@ -5190,7 +5191,7 @@ func TestBackupAzureAccountName(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() values := url.Values{} @@ -5217,15 +5218,15 @@ func TestPointInTimeRecovery(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1000 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - fullBackupDir := LocalFoo + "/full" + fullBackupDir := localFoo + "/full" sqlDB.Exec(t, `BACKUP data.* TO $1`, fullBackupDir) sqlDB.Exec(t, `UPDATE data.bank SET balance = 2`) - incBackupDir := LocalFoo + "/inc" + incBackupDir := localFoo + "/inc" sqlDB.Exec(t, `BACKUP data.* TO $1 INCREMENTAL FROM $2`, incBackupDir, fullBackupDir) var beforeBadThingTs string @@ -5246,7 +5247,7 @@ func TestPointInTimeRecovery(t *testing.T) { // RENAME-ing the table into the final location. t.Run("recovery=new-backup", func(t *testing.T) { sqlDB = sqlutils.MakeSQLRunner(sqlDB.DB) - recoveryDir := LocalFoo + "/new-backup" + recoveryDir := localFoo + "/new-backup" sqlDB.Exec(t, fmt.Sprintf(`BACKUP data.* TO $1 AS OF SYSTEM TIME '%s'`, beforeBadThingTs), recoveryDir, @@ -5268,7 +5269,7 @@ func TestPointInTimeRecovery(t *testing.T) { // using that. Everything else works the same as above. t.Run("recovery=inc-backup", func(t *testing.T) { sqlDB = sqlutils.MakeSQLRunner(sqlDB.DB) - recoveryDir := LocalFoo + "/inc-backup" + recoveryDir := localFoo + "/inc-backup" sqlDB.Exec(t, fmt.Sprintf(`BACKUP data.* TO $1 AS OF SYSTEM TIME '%s' INCREMENTAL FROM $2, $3`, beforeBadThingTs), recoveryDir, fullBackupDir, incBackupDir, @@ -5294,7 +5295,7 @@ func TestBackupRestoreDropDB(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `DROP DATABASE data`) @@ -5302,9 +5303,9 @@ func TestBackupRestoreDropDB(t *testing.T) { sqlDB.Exec(t, `CREATE TABLE data.bank (i int)`) sqlDB.Exec(t, `INSERT INTO data.bank VALUES (1)`) - sqlDB.Exec(t, "BACKUP DATABASE data TO $1", LocalFoo) + sqlDB.Exec(t, "BACKUP DATABASE data TO $1", localFoo) sqlDB.Exec(t, "CREATE DATABASE data2") - sqlDB.Exec(t, "RESTORE data.* FROM $1 WITH OPTIONS (into_db='data2')", LocalFoo) + sqlDB.Exec(t, "RESTORE data.* FROM $1 WITH OPTIONS (into_db='data2')", localFoo) expected := sqlDB.QueryStr(t, `SELECT * FROM data.bank`) sqlDB.CheckQueryResults(t, `SELECT * FROM data2.bank`, expected) @@ -5315,7 +5316,7 @@ func TestBackupRestoreDropTable(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `DROP TABLE data.bank`) @@ -5324,9 +5325,9 @@ func TestBackupRestoreDropTable(t *testing.T) { INSERT INTO data.bank VALUES (1); `) - sqlDB.Exec(t, "BACKUP DATABASE data TO $1", LocalFoo) + sqlDB.Exec(t, "BACKUP DATABASE data TO $1", localFoo) sqlDB.Exec(t, "CREATE DATABASE data2") - sqlDB.Exec(t, "RESTORE data.* FROM $1 WITH OPTIONS (into_db='data2')", LocalFoo) + sqlDB.Exec(t, "RESTORE data.* FROM $1 WITH OPTIONS (into_db='data2')", localFoo) expected := sqlDB.QueryStr(t, `SELECT * FROM data.bank`) sqlDB.CheckQueryResults(t, `SELECT * FROM data2.bank`, expected) @@ -5337,11 +5338,11 @@ func TestBackupRestoreIncrementalAddTable(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE data2`) sqlDB.Exec(t, `CREATE TABLE data.t (s string PRIMARY KEY)`) - full, inc := LocalFoo+"/full", LocalFoo+"/inc" + full, inc := localFoo+"/full", localFoo+"/inc" sqlDB.Exec(t, `INSERT INTO data.t VALUES ('before')`) sqlDB.Exec(t, `BACKUP data.*, data2.* TO $1`, full) @@ -5356,11 +5357,11 @@ func TestBackupRestoreIncrementalAddTableMissing(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE data2`) sqlDB.Exec(t, `CREATE TABLE data.t (s string PRIMARY KEY)`) - full, inc := LocalFoo+"/full", LocalFoo+"/inc" + full, inc := localFoo+"/full", localFoo+"/inc" sqlDB.Exec(t, `INSERT INTO data.t VALUES ('before')`) sqlDB.Exec(t, `BACKUP data.* TO $1`, full) @@ -5378,10 +5379,10 @@ func TestBackupRestoreIncrementalTrucateTable(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE TABLE data.t (s string PRIMARY KEY)`) - full, inc := LocalFoo+"/full", LocalFoo+"/inc" + full, inc := localFoo+"/full", localFoo+"/inc" sqlDB.Exec(t, `INSERT INTO data.t VALUES ('before')`) sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, full) @@ -5396,10 +5397,10 @@ func TestBackupRestoreIncrementalDropTable(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE TABLE data.t (s string PRIMARY KEY)`) - full, inc := LocalFoo+"/full", LocalFoo+"/inc" + full, inc := localFoo+"/full", localFoo+"/inc" sqlDB.Exec(t, `INSERT INTO data.t VALUES ('before')`) sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, full) @@ -5424,12 +5425,12 @@ func TestFileIOLimits(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 11 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() elsewhere := "nodelocal://0/../../blah" - sqlDB.Exec(t, `BACKUP data.bank TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank TO $1`, localFoo) sqlDB.ExpectErr( t, "local file access to paths outside of external-io-dir is not allowed", `BACKUP data.bank TO $1`, elsewhere, @@ -5437,7 +5438,7 @@ func TestFileIOLimits(t *testing.T) { sqlDB.Exec(t, `DROP TABLE data.bank`) - sqlDB.Exec(t, `RESTORE data.bank FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.bank FROM $1`, localFoo) sqlDB.ExpectErr( t, "local file access to paths outside of external-io-dir is not allowed", `RESTORE data.bank FROM $1`, elsewhere, @@ -5461,7 +5462,7 @@ func TestDetachedBackup(t *testing.T) { const numAccounts = 1 ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() db := sqlDB.DB.(*gosql.DB) @@ -5469,18 +5470,18 @@ func TestDetachedBackup(t *testing.T) { // running backup under transaction requires DETACHED. var jobID jobspb.JobID err := crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error { - return tx.QueryRow(`BACKUP DATABASE data TO $1`, LocalFoo).Scan(&jobID) + return tx.QueryRow(`BACKUP DATABASE data TO $1`, localFoo).Scan(&jobID) }) require.True(t, testutils.IsError(err, "BACKUP cannot be used inside a transaction without DETACHED option")) // Okay to run DETACHED backup, even w/out explicit transaction. - sqlDB.QueryRow(t, `BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo).Scan(&jobID) + sqlDB.QueryRow(t, `BACKUP DATABASE data TO $1 WITH DETACHED`, localFoo).Scan(&jobID) waitForSuccessfulJob(t, tc, jobID) // Backup again, under explicit transaction. err = crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error { - return tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo+"/1").Scan(&jobID) + return tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, localFoo+"/1").Scan(&jobID) }) require.NoError(t, err) waitForSuccessfulJob(t, tc, jobID) @@ -5492,7 +5493,7 @@ func TestDetachedBackup(t *testing.T) { tx, err := db.Begin() require.NoError(t, err) err = crdb.Execute(func() error { - return tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, LocalFoo+"/2").Scan(&jobID) + return tx.QueryRow(`BACKUP DATABASE data TO $1 WITH DETACHED`, localFoo+"/2").Scan(&jobID) }) require.NoError(t, err) require.NoError(t, tx.Rollback()) @@ -5500,7 +5501,7 @@ func TestDetachedBackup(t *testing.T) { // Ensure that we can backup again to the same location as the backup that was // rolledback. - sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, LocalFoo+"/2") + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo+"/2") } func TestDetachedRestore(t *testing.T) { @@ -5509,7 +5510,7 @@ func TestDetachedRestore(t *testing.T) { const numAccounts = 1 ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() db := sqlDB.DB.(*gosql.DB) @@ -5517,26 +5518,26 @@ func TestDetachedRestore(t *testing.T) { // Run a BACKUP. sqlDB.Exec(t, `CREATE TABLE data.t (id INT, name STRING)`) sqlDB.Exec(t, `INSERT INTO data.t VALUES (1, 'foo'), (2, 'bar')`) - sqlDB.Exec(t, `BACKUP TABLE data.t TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TABLE data.t TO $1`, localFoo) sqlDB.Exec(t, `CREATE DATABASE test`) // Running RESTORE under transaction requires DETACHED. var jobID jobspb.JobID err := crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error { - return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH INTO_DB=test`, LocalFoo).Scan(&jobID) + return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH INTO_DB=test`, localFoo).Scan(&jobID) }) require.True(t, testutils.IsError(err, "RESTORE cannot be used inside a transaction without DETACHED option")) // Okay to run DETACHED RESTORE, even w/out explicit transaction. sqlDB.QueryRow(t, `RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, - LocalFoo).Scan(&jobID) + localFoo).Scan(&jobID) waitForSuccessfulJob(t, tc, jobID) sqlDB.Exec(t, `DROP TABLE test.t`) // RESTORE again, under explicit transaction. err = crdb.ExecuteTx(ctx, db, nil /* txopts */, func(tx *gosql.Tx) error { - return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, LocalFoo).Scan(&jobID) + return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, localFoo).Scan(&jobID) }) require.NoError(t, err) waitForSuccessfulJob(t, tc, jobID) @@ -5549,7 +5550,7 @@ func TestDetachedRestore(t *testing.T) { tx, err := db.Begin() require.NoError(t, err) err = crdb.Execute(func() error { - return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, LocalFoo).Scan(&jobID) + return tx.QueryRow(`RESTORE TABLE t FROM $1 WITH DETACHED, INTO_DB=test`, localFoo).Scan(&jobID) }) require.NoError(t, err) require.NoError(t, tx.Rollback()) @@ -5560,11 +5561,11 @@ func TestBackupRestoreSequence(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) const numAccounts = 1 - _, origDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: dir} - backupLoc := LocalFoo + backupLoc := localFoo origDB.Exec(t, `CREATE SEQUENCE data.t_id_seq`) origDB.Exec(t, `CREATE TABLE data.t (id INT PRIMARY KEY DEFAULT nextval('data.t_id_seq'), v text)`) @@ -5628,10 +5629,10 @@ func TestBackupRestoreSequence(t *testing.T) { newDB.ExpectErr( t, "pq: cannot restore table \"t\" without referenced sequence \\d+ \\(or \"skip_missing_sequences\" option\\)", - `RESTORE TABLE t FROM $1`, LocalFoo, + `RESTORE TABLE t FROM $1`, localFoo, ) - newDB.Exec(t, `RESTORE TABLE t FROM $1 WITH OPTIONS (skip_missing_sequences)`, LocalFoo) + newDB.Exec(t, `RESTORE TABLE t FROM $1 WITH OPTIONS (skip_missing_sequences)`, localFoo) // Verify that the table was restored correctly. newDB.CheckQueryResults(t, `SELECT * FROM data.t`, [][]string{ @@ -5679,7 +5680,7 @@ func TestBackupRestoreSequencesInViews(t *testing.T) { // Test backing up and restoring a database with views referencing sequences. t.Run("database", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE d`) @@ -5713,7 +5714,7 @@ func TestBackupRestoreSequencesInViews(t *testing.T) { // Test backing up and restoring both view and sequence. t.Run("restore view and sequence", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE d`) @@ -5750,7 +5751,7 @@ func TestBackupRestoreSequencesInViews(t *testing.T) { // Test backing up and restoring just the view. t.Run("restore just the view", func(t *testing.T) { - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE d`) @@ -5774,12 +5775,12 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, origDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() args := base.TestServerArgs{ExternalIODir: dir} // Setup for sequence ownership backup/restore tests in the same database. - backupLoc := LocalFoo + `/d` + backupLoc := localFoo + `/d` origDB.Exec(t, "SET CLUSTER SETTING sql.cross_db_sequence_owners.enabled = TRUE") origDB.Exec(t, `CREATE DATABASE d`) origDB.Exec(t, `CREATE TABLE d.t(a int)`) @@ -5900,7 +5901,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { }) // Setup for cross-database ownership backup-restore tests. - backupLocD2D3 := LocalFoo + `/d2d3` + backupLocD2D3 := localFoo + `/d2d3` origDB.Exec(t, `CREATE DATABASE d2`) origDB.Exec(t, `CREATE TABLE d2.t(a int)`) @@ -6031,13 +6032,13 @@ func TestBackupRestoreShowJob(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP DATABASE data TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE data TO $1 WITH revision_history`, localFoo) sqlDB.Exec(t, `CREATE DATABASE "data 2"`) - sqlDB.Exec(t, `RESTORE data.bank FROM $1 WITH skip_missing_foreign_keys, into_db = $2`, LocalFoo, "data 2") + sqlDB.Exec(t, `RESTORE data.bank FROM $1 WITH skip_missing_foreign_keys, into_db = $2`, localFoo, "data 2") // The "updating privileges" clause in the SELECT statement is for excluding jobs // run by an unrelated startup migration. // TODO (lucy): Update this if/when we decide to change how these jobs queued by @@ -6056,7 +6057,7 @@ func TestBackupCreatedStats(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled=false`) @@ -6065,10 +6066,10 @@ func TestBackupCreatedStats(t *testing.T) { injectStats(t, sqlDB, "data.bank", "id") injectStats(t, sqlDB, "data.foo", "a") - sqlDB.Exec(t, `BACKUP data.bank, data.foo TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank, data.foo TO $1 WITH revision_history`, localFoo) sqlDB.Exec(t, `CREATE DATABASE "data 2"`) sqlDB.Exec(t, `RESTORE data.bank, data.foo FROM $1 WITH skip_missing_foreign_keys, into_db = $2`, - LocalFoo, "data 2") + localFoo, "data 2") sqlDB.CheckQueryResults(t, getStatsQuery(`"data 2".bank`), sqlDB.QueryStr(t, getStatsQuery("data.bank"))) @@ -6082,13 +6083,13 @@ func TestBackupRestoreEmptyDB(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE empty`) - sqlDB.Exec(t, `BACKUP DATABASE empty TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP DATABASE empty TO $1`, localFoo) sqlDB.Exec(t, `DROP DATABASE empty`) - sqlDB.Exec(t, `RESTORE DATABASE empty FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE DATABASE empty FROM $1`, localFoo) sqlDB.CheckQueryResults(t, `USE empty; SHOW TABLES;`, [][]string{}) } @@ -6097,7 +6098,7 @@ func TestBackupRestoreSubsetCreatedStats(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled=false`) @@ -6106,13 +6107,13 @@ func TestBackupRestoreSubsetCreatedStats(t *testing.T) { bankStats := injectStats(t, sqlDB, "data.bank", "id") injectStats(t, sqlDB, "data.foo", "a") - sqlDB.Exec(t, `BACKUP data.bank, data.foo TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank, data.foo TO $1 WITH revision_history`, localFoo) // Clear the stats. sqlDB.Exec(t, `DELETE FROM system.table_statistics WHERE true`) sqlDB.Exec(t, `CREATE DATABASE "data 2"`) sqlDB.Exec(t, `CREATE TABLE "data 2".foo (a INT)`) sqlDB.Exec(t, `RESTORE data.bank FROM $1 WITH skip_missing_foreign_keys, into_db = $2`, - LocalFoo, "data 2") + localFoo, "data 2") // Ensure that bank's stats have been restored, but foo's have not. sqlDB.CheckQueryResults(t, getStatsQuery(`"data 2".bank`), bankStats) @@ -6129,7 +6130,7 @@ func TestBackupHandlesDroppedTypeStatsCollection(t *testing.T) { const dest = "userfile:///basefoo" const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE TYPE greeting as ENUM ('hello')`) @@ -6159,7 +6160,7 @@ func TestBatchedInsertStats(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() ctx := context.Background() @@ -6245,7 +6246,7 @@ func TestBackupRestoreCorruptedStatsIgnored(t *testing.T) { const dest = "userfile:///basefoo" const numAccounts = 1 - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() @@ -6285,7 +6286,7 @@ func TestBackupCreatedStatsFromIncrementalBackup(t *testing.T) { const incremental1Foo = "nodelocal://0/incremental1foo" const incremental2Foo = "nodelocal://0/incremental2foo" const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() var beforeTs string @@ -6293,22 +6294,22 @@ func TestBackupCreatedStatsFromIncrementalBackup(t *testing.T) { // Create the 1st backup, with stats estimating 50 rows. injectStatsWithRowCount(t, sqlDB, "data.bank", "id", 50 /* rowCount */) - sqlDB.Exec(t, `BACKUP data.bank TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank TO $1 WITH revision_history`, localFoo) // Create the 2nd backup, with stats estimating 100 rows. injectStatsWithRowCount(t, sqlDB, "data.bank", "id", 100 /* rowCount */) statsBackup2 := sqlDB.QueryStr(t, getStatsQuery("data.bank")) sqlDB.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&beforeTs) // Save time to restore to this point. - sqlDB.Exec(t, `BACKUP data.bank TO $1 INCREMENTAL FROM $2 WITH revision_history`, incremental1Foo, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank TO $1 INCREMENTAL FROM $2 WITH revision_history`, incremental1Foo, localFoo) // Create the 3rd backup, with stats estimating 500 rows. injectStatsWithRowCount(t, sqlDB, "data.bank", "id", 500 /* rowCount */) - sqlDB.Exec(t, `BACKUP data.bank TO $1 INCREMENTAL FROM $2, $3 WITH revision_history`, incremental2Foo, LocalFoo, incremental1Foo) + sqlDB.Exec(t, `BACKUP data.bank TO $1 INCREMENTAL FROM $2, $3 WITH revision_history`, incremental2Foo, localFoo, incremental1Foo) // Restore the 2nd backup. sqlDB.Exec(t, `CREATE DATABASE "data 2"`) sqlDB.Exec(t, fmt.Sprintf(`RESTORE data.bank FROM "%s", "%s", "%s" AS OF SYSTEM TIME %s WITH skip_missing_foreign_keys, into_db = "%s"`, - LocalFoo, incremental1Foo, incremental2Foo, beforeTs, "data 2")) + localFoo, incremental1Foo, incremental2Foo, beforeTs, "data 2")) // Expect the stats look as they did in the second backup. sqlDB.CheckQueryResults(t, getStatsQuery(`"data 2".bank`), statsBackup2) @@ -6778,23 +6779,6 @@ func TestPublicIndexTableSpans(t *testing.T) { } } -func getFirstStoreReplica( - t *testing.T, s serverutils.TestServerInterface, key roachpb.Key, -) (*kvserver.Store, *kvserver.Replica) { - t.Helper() - store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) - require.NoError(t, err) - var repl *kvserver.Replica - testutils.SucceedsSoon(t, func() error { - repl = store.LookupReplica(roachpb.RKey(key)) - if repl == nil { - return errors.New(`could not find replica`) - } - return nil - }) - return store, repl -} - // TestRestoreJobErrorPropagates ensures that errors from creating the job // record propagate correctly. func TestRestoreErrorPropagates(t *testing.T) { @@ -7068,7 +7052,7 @@ func TestBackupRestoreInsideTenant(t *testing.T) { } const numAccounts = 1 - tc, systemDB, dir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, systemDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) _, _ = tc, systemDB defer cleanupFn() srv := tc.Server(0) @@ -7463,11 +7447,11 @@ func TestClientDisconnect(t *testing.T) { }{ { jobType: "BACKUP", - jobCommand: fmt.Sprintf("BACKUP TO '%s'", LocalFoo), + jobCommand: fmt.Sprintf("BACKUP TO '%s'", localFoo), }, { jobType: "RESTORE", - jobCommand: fmt.Sprintf("RESTORE data.* FROM '%s' WITH into_db='%s'", LocalFoo, restoreDB), + jobCommand: fmt.Sprintf("RESTORE data.* FROM '%s' WITH into_db='%s'", localFoo, restoreDB), }, } @@ -7506,7 +7490,7 @@ func TestClientDisconnect(t *testing.T) { }, }} args.ServerArgs.Knobs = knobs - tc, sqlDB, _, cleanup := backupRestoreTestSetupWithParams(t, MultiNode, 1 /* numAccounts */, InitManualReplication, args) + tc, sqlDB, _, cleanup := backupRestoreTestSetupWithParams(t, multiNode, 1 /* numAccounts */, InitManualReplication, args) defer cleanup() ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -7518,7 +7502,7 @@ func TestClientDisconnect(t *testing.T) { if testCase.jobType == "RESTORE" { close(allowResponse) sqlDB.Exec(t, fmt.Sprintf("CREATE DATABASE %s", restoreDB)) - sqlDB.Exec(t, "BACKUP TO $1", LocalFoo) + sqlDB.Exec(t, "BACKUP TO $1", localFoo) // Reset the channels. There will be a request on the gotRequest channel // due to the backup. allowResponse = make(chan struct{}) @@ -7629,7 +7613,7 @@ func TestBackupDoesNotHangOnIntent(t *testing.T) { const numAccounts = 10 ctx := context.Background() - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, "SET CLUSTER SETTING bulkio.backup.read_with_priority_after = '100ms'") @@ -7664,7 +7648,7 @@ func TestRestoreTypeDescriptorsRollBack(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() for _, server := range tc.Servers { @@ -7716,7 +7700,7 @@ func TestRestoreResetsDescriptorVersions(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() kvDB := tc.Server(0).DB() @@ -7792,7 +7776,7 @@ func TestOfflineDescriptorsDuringRestore(t *testing.T) { t.Run("restore-database", func(t *testing.T) { ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -7885,7 +7869,7 @@ CREATE TYPE sc.typ AS ENUM ('hello'); t.Run("restore-into-existing-database", func(t *testing.T) { ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -8031,7 +8015,7 @@ func TestCleanupDoesNotDeleteParentsWithChildObjects(t *testing.T) { t.Run("clean-up-database-with-schema", func(t *testing.T) { ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -8093,7 +8077,7 @@ func TestCleanupDoesNotDeleteParentsWithChildObjects(t *testing.T) { t.Run("clean-up-database-with-table", func(t *testing.T) { ctx := context.Background() - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -8154,7 +8138,7 @@ func TestCleanupDoesNotDeleteParentsWithChildObjects(t *testing.T) { t.Run("clean-up-schema-with-table", func(t *testing.T) { ctx := context.Background() - tc, _, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -8225,7 +8209,7 @@ func TestCleanupDoesNotDeleteParentsWithChildObjects(t *testing.T) { func TestManifestTooNew(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, rawDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE r1`) sqlDB.Exec(t, `BACKUP DATABASE r1 TO 'nodelocal://0/too_new'`) @@ -8290,7 +8274,7 @@ func TestManifestTooNew(t *testing.T) { func TestManifestBitFlip(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, rawDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE r1; CREATE DATABASE r2; CREATE DATABASE r3;`) const checksumError = "checksum mismatch" @@ -8589,7 +8573,7 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { kvDB := tc.Server(0).DB() locationToDir := func(location string) string { - return strings.Replace(location, LocalFoo, filepath.Join(rawDir, "foo"), 1) + return strings.Replace(location, localFoo, filepath.Join(rawDir, "foo"), 1) } // Test timeline: @@ -8603,7 +8587,7 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { // 8. Inc 4 // First take a full backup. - fullBackup := LocalFoo + "/full" + fullBackup := localFoo + "/full" sqlDB.Exec(t, `BACKUP DATABASE data TO $1 WITH revision_history`, fullBackup) var dataBankTableID descpb.ID sqlDB.QueryRow(t, `SELECT 'data.bank'::regclass::int`). @@ -8623,8 +8607,8 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { return errors.Wrap(err, "creating index") }) - inc1Loc := LocalFoo + "/inc1" - inc2Loc := LocalFoo + "/inc2" + inc1Loc := localFoo + "/inc1" + inc2Loc := localFoo + "/inc2" g.Go(func() error { defer backfillBlockers[0].allowToProceed() @@ -8661,7 +8645,7 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { // Take another incremental backup that should only contain the newly added // index. - inc3Loc := LocalFoo + "/inc3" + inc3Loc := localFoo + "/inc3" sqlDB.Exec(t, `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2, $3, $4 WITH revision_history`, inc3Loc, fullBackup, inc1Loc, inc2Loc) inc3Spans := getSpansFromManifest(ctx, t, locationToDir(inc3Loc)) @@ -8672,7 +8656,7 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { sqlDB.Exec(t, `DROP INDEX new_balance_idx`) // Take another incremental backup. - inc4Loc := LocalFoo + "/inc4" + inc4Loc := localFoo + "/inc4" sqlDB.Exec(t, `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2, $3, $4, $5 WITH revision_history`, inc4Loc, fullBackup, inc1Loc, inc2Loc, inc3Loc) @@ -8766,7 +8750,7 @@ func TestBackupWorkerFailure(t *testing.T) { const numAccounts = 100 - tc, _, _, cleanup := backupRestoreTestSetupWithParams(t, MultiNode, numAccounts, + tc, _, _, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, InitManualReplication, params) conn := tc.Conns[0] sqlDB := sqlutils.MakeSQLRunner(conn) @@ -8816,7 +8800,7 @@ func TestSpanMergingBeforeGCThreshold(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() kvDB := tc.Server(0).DB() @@ -9001,7 +8985,7 @@ func TestRestorePauseOnError(t *testing.T) { func TestDroppedDescriptorRevisionAndSystemDBIDClash(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` @@ -9012,12 +8996,12 @@ DROP TABLE foo; var aost string sqlDB.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&aost) - sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, localFoo) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupEmptyCluster() - sqlDBRestore.Exec(t, "RESTORE FROM $1 AS OF SYSTEM TIME "+aost, LocalFoo) + sqlDBRestore.Exec(t, "RESTORE FROM $1 AS OF SYSTEM TIME "+aost, localFoo) } // TestRestoreNewDatabaseName tests the new_db_name optional feature for single database @@ -9027,7 +9011,7 @@ func TestRestoreNewDatabaseName(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE fkdb`) @@ -9036,29 +9020,29 @@ func TestRestoreNewDatabaseName(t *testing.T) { for i := 0; i < 10; i++ { sqlDB.Exec(t, `INSERT INTO fkdb.fk (ind) VALUES ($1)`, i) } - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) // Ensure restore fails with new_db_name on cluster, table, and multiple database restores t.Run("new_db_name syntax checks", func(t *testing.T) { expectedErr := "new_db_name can only be used for RESTORE DATABASE with a single target database" - sqlDB.ExpectErr(t, expectedErr, "RESTORE FROM $1 with new_db_name = 'new_fkdb'", LocalFoo) + sqlDB.ExpectErr(t, expectedErr, "RESTORE FROM $1 with new_db_name = 'new_fkdb'", localFoo) sqlDB.ExpectErr(t, expectedErr, "RESTORE DATABASE fkdb, "+ - "data FROM $1 with new_db_name = 'new_fkdb'", LocalFoo) + "data FROM $1 with new_db_name = 'new_fkdb'", localFoo) sqlDB.ExpectErr(t, expectedErr, "RESTORE TABLE fkdb.fk FROM $1 with new_db_name = 'new_fkdb'", - LocalFoo) + localFoo) }) // Should fail because 'fkbd' database is still in cluster sqlDB.ExpectErr(t, `database "fkdb" already exists`, - "RESTORE DATABASE fkdb FROM $1", LocalFoo) + "RESTORE DATABASE fkdb FROM $1", localFoo) // Should pass because 'new_fkdb' is not in cluster - sqlDB.Exec(t, "RESTORE DATABASE fkdb FROM $1 WITH new_db_name = 'new_fkdb'", LocalFoo) + sqlDB.Exec(t, "RESTORE DATABASE fkdb FROM $1 WITH new_db_name = 'new_fkdb'", localFoo) // Verify restored database is in cluster with new name sqlDB.CheckQueryResults(t, @@ -9071,7 +9055,7 @@ func TestRestoreNewDatabaseName(t *testing.T) { // Should fail because we just restored new_fkbd into cluster sqlDB.ExpectErr(t, `database "new_fkdb" already exists`, - "RESTORE DATABASE fkdb FROM $1 WITH new_db_name = 'new_fkdb'", LocalFoo) + "RESTORE DATABASE fkdb FROM $1 WITH new_db_name = 'new_fkdb'", localFoo) } // TestRestoreRemappingOfExistingUDTInColExpr is a regression test for a nil @@ -9083,7 +9067,7 @@ func TestRestoreRemappingOfExistingUDTInColExpr(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, ` @@ -9121,7 +9105,8 @@ func TestGCDropIndexSpanExpansion(t *testing.T) { }, }}) defer tc.Stopper().Stop(ctx) - sqlRunner := sqlutils.MakeSQLRunner(tc.Conns[0]) + conn := tc.Conns[0] + sqlRunner := sqlutils.MakeSQLRunner(conn) sqlRunner.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) // speeds up the test sqlRunner.Exec(t, ` @@ -9148,23 +9133,7 @@ DROP INDEX foo@bar; // Wait for the GC to complete. jobutils.WaitForJob(t, sqlRunner, gcJobID) - - waitForTableSplit := func() { - testutils.SucceedsSoon(t, func() error { - count := 0 - sqlRunner.QueryRow(t, - "SELECT count(*) "+ - "FROM crdb_internal.ranges_no_leases "+ - "WHERE table_name = $1 "+ - "AND database_name = $2", - "foo", "test").Scan(&count) - if count == 0 { - return errors.New("waiting for table split") - } - return nil - }) - } - waitForTableSplit() + waitForTableSplit(t, conn, "foo", "test") // This backup should succeed since the spans being backed up have a default // GC TTL of 25 hours. @@ -9181,7 +9150,7 @@ func TestRestoreSchemaDescriptorsRollBack(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() for _, server := range tc.Servers { @@ -9265,7 +9234,7 @@ func TestBackupRestoreSeparateIncrementalPrefix(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() const c1, c2, c3 = `nodelocal://0/full/`, `nodelocal://1/full/`, `nodelocal://2/full/` @@ -9326,3 +9295,164 @@ func TestBackupRestoreSeparateIncrementalPrefix(t *testing.T) { sqlDB.Exec(t, "DROP DATABASE inc_fkdb;") } } + +func TestExcludeDataFromBackupAndRestore(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + tc, sqlDB, iodir, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, 10, + InitManualReplication, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test + SpanConfig: &spanconfig.TestingKnobs{ + SQLWatcherCheckpointNoopsEveryDurationOverride: 100 * time.Millisecond, + }, + }, + }, + }) + defer cleanupFn() + + _, restoreDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, iodir, InitManualReplication, + base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test + }, + }, + }) + defer cleanup() + + sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + sqlDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + conn := tc.Conns[0] + + sqlDB.Exec(t, `CREATE TABLE data.foo (id INT, INDEX bar(id))`) + sqlDB.Exec(t, `INSERT INTO data.foo select * from generate_series(1,10)`) + + // Create another table. + sqlDB.Exec(t, `CREATE TABLE data.bar (id INT, INDEX bar(id))`) + sqlDB.Exec(t, `INSERT INTO data.bar select * from generate_series(1,10)`) + + // Set foo to exclude_data_from_backup and back it up. The ExportRequest + // should be a noop and backup no data. + sqlDB.Exec(t, `ALTER TABLE data.foo SET (exclude_data_from_backup = true)`) + waitForTableSplit(t, conn, "foo", "data") + waitForReplicaFieldToBeSet(t, tc, conn, "foo", "data", func(r *kvserver.Replica) (bool, error) { + if !r.ExcludeDataFromBackup() { + return false, errors.New("waiting for exclude_data_from_backup to be applied") + } + return true, nil + }) + waitForTableSplit(t, conn, "bar", "data") + sqlDB.Exec(t, `BACKUP DATABASE data TO $1`, localFoo) + + restoreDB.Exec(t, `RESTORE DATABASE data FROM $1`, localFoo) + require.Len(t, restoreDB.QueryStr(t, `SELECT * FROM data.foo`), 0) + require.Len(t, restoreDB.QueryStr(t, `SELECT * FROM data.bar`), 10) +} + +// TestExportRequestBelowGCThresholdOnDataExcludedFromBackup tests that a +// `BatchTimestampBeforeGCError` on an ExportRequest targeting a table that has +// been marked as excluded from backup, does not cause the backup to fail. +func TestExportRequestBelowGCThresholdOnDataExcludedFromBackup(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderStressRace(t, "test is too slow to run under race") + + ctx := context.Background() + localExternalDir, cleanup := testutils.TempDir(t) + defer cleanup() + args := base.TestClusterArgs{} + args.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ + DisableGCQueue: true, + DisableLastProcessedCheck: true, + } + args.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() + args.ServerArgs.ExternalIODir = localExternalDir + tc := testcluster.StartTestCluster(t, 3, args) + defer tc.Stopper().Stop(ctx) + + tc.WaitForNodeLiveness(t) + require.NoError(t, tc.WaitForFullReplication()) + + for _, server := range tc.Servers { + registry := server.JobRegistry().(*jobs.Registry) + registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{ + jobspb.TypeBackup: func(raw jobs.Resumer) jobs.Resumer { + r := raw.(*backupResumer) + r.testingKnobs.ignoreProtectedTimestamps = true + return r + }, + } + } + conn := tc.ServerConn(0) + _, err := conn.Exec("CREATE TABLE foo (k INT PRIMARY KEY, v BYTES)") + require.NoError(t, err) + + _, err = conn.Exec("SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms';") + require.NoError(t, err) + + _, err = conn.Exec("SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'") // speeds up the test + require.NoError(t, err) + + const tableRangeMaxBytes = 1 << 18 + _, err = conn.Exec("ALTER TABLE foo CONFIGURE ZONE USING "+ + "gc.ttlseconds = 1, range_max_bytes = $1, range_min_bytes = 1<<10;", tableRangeMaxBytes) + require.NoError(t, err) + + rRand, _ := randutil.NewTestRand() + upsertUntilBackpressure := func() { + for { + _, err := conn.Exec("UPSERT INTO foo VALUES (1, $1)", + randutil.RandBytes(rRand, 1<<15)) + if testutils.IsError(err, "backpressure") { + break + } + require.NoError(t, err) + } + } + const processedPattern = `(?s)shouldQueue=true.*processing replica.*GC score after GC` + processedRegexp := regexp.MustCompile(processedPattern) + + gcSoon := func() { + testutils.SucceedsSoon(t, func() error { + upsertUntilBackpressure() + s, repl := getStoreAndReplica(t, tc, conn, "foo", "defaultdb") + trace, _, err := s.ManuallyEnqueue(ctx, "mvccGC", repl, false) + require.NoError(t, err) + if !processedRegexp.MatchString(trace.String()) { + return errors.Errorf("%q does not match %q", trace.String(), processedRegexp) + } + return nil + }) + } + + waitForTableSplit(t, conn, "foo", "defaultdb") + waitForReplicaFieldToBeSet(t, tc, conn, "foo", "defaultdb", func(r *kvserver.Replica) (bool, error) { + if r.GetMaxBytes() != tableRangeMaxBytes { + return false, errors.New("waiting for range_max_bytes to be applied") + } + return true, nil + }) + + var tsBefore string + require.NoError(t, conn.QueryRow("SELECT cluster_logical_timestamp()").Scan(&tsBefore)) + gcSoon() + + _, err = conn.Exec(fmt.Sprintf("BACKUP TABLE foo TO $1 AS OF SYSTEM TIME '%s'", tsBefore), localFoo) + testutils.IsError(err, "must be after replica GC threshold") + + _, err = conn.Exec(`ALTER TABLE foo SET (exclude_data_from_backup = true)`) + require.NoError(t, err) + waitForReplicaFieldToBeSet(t, tc, conn, "foo", "defaultdb", func(r *kvserver.Replica) (bool, error) { + if !r.ExcludeDataFromBackup() { + return false, errors.New("waiting for exclude_data_from_backup to be applied") + } + return true, nil + }) + + _, err = conn.Exec(fmt.Sprintf("BACKUP TABLE foo TO $1 AS OF SYSTEM TIME '%s'", tsBefore), localFoo) + require.NoError(t, err) +} diff --git a/pkg/ccl/backupccl/bench_test.go b/pkg/ccl/backupccl/bench_test.go index f07e6c32bf72..934d9bc02d81 100644 --- a/pkg/ccl/backupccl/bench_test.go +++ b/pkg/ccl/backupccl/bench_test.go @@ -6,13 +6,12 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package backupccl_test +package backupccl import ( "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl/sampledataccl" "github.com/cockroachdb/cockroach/pkg/workload/bank" ) @@ -22,8 +21,8 @@ func BenchmarkDatabaseBackup(b *testing.B) { // documentation's description. We're getting useful information out of it, // but this is not a pattern to cargo-cult. - _, sqlDB, dir, cleanupFn := backupccl.BackupRestoreTestSetup(b, backupccl.MultiNode, - 0 /* numAccounts */, backupccl.InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(b, multiNode, + 0 /* numAccounts */, InitManualReplication) defer cleanupFn() sqlDB.Exec(b, `DROP TABLE data.bank`) @@ -44,7 +43,7 @@ func BenchmarkDatabaseBackup(b *testing.B) { b.ResetTimer() var unused string var dataSize int64 - sqlDB.QueryRow(b, fmt.Sprintf(`BACKUP DATABASE data TO '%s'`, backupccl.LocalFoo)).Scan( + sqlDB.QueryRow(b, fmt.Sprintf(`BACKUP DATABASE data TO '%s'`, localFoo)).Scan( &unused, &unused, &unused, &unused, &unused, &dataSize, ) b.StopTimer() @@ -56,8 +55,8 @@ func BenchmarkDatabaseRestore(b *testing.B) { // documentation's description. We're getting useful information out of it, // but this is not a pattern to cargo-cult. - _, sqlDB, dir, cleanup := backupccl.BackupRestoreTestSetup(b, backupccl.MultiNode, - 0 /* numAccounts*/, backupccl.InitManualReplication) + _, sqlDB, dir, cleanup := backupRestoreTestSetup(b, multiNode, + 0 /* numAccounts*/, InitManualReplication) defer cleanup() sqlDB.Exec(b, `DROP TABLE data.bank`) @@ -74,12 +73,12 @@ func BenchmarkDatabaseRestore(b *testing.B) { func BenchmarkEmptyIncrementalBackup(b *testing.B) { const numStatements = 100000 - _, sqlDB, dir, cleanupFn := backupccl.BackupRestoreTestSetup(b, backupccl.MultiNode, - 0 /* numAccounts */, backupccl.InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(b, multiNode, + 0 /* numAccounts */, InitManualReplication) defer cleanupFn() - restoreURI := backupccl.LocalFoo + "/restore" - fullURI := backupccl.LocalFoo + "/full" + restoreURI := localFoo + "/restore" + fullURI := localFoo + "/full" bankData := bank.FromRows(numStatements).Tables()[0] _, err := sampledataccl.ToBackup(b, bankData, dir, "foo/restore") @@ -100,7 +99,7 @@ func BenchmarkEmptyIncrementalBackup(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - incrementalDir := backupccl.LocalFoo + fmt.Sprintf("/incremental%d", i) + incrementalDir := localFoo + fmt.Sprintf("/incremental%d", i) sqlDB.Exec(b, `BACKUP DATABASE data TO $1 INCREMENTAL FROM $2`, incrementalDir, fullURI) } b.StopTimer() @@ -113,12 +112,12 @@ func BenchmarkEmptyIncrementalBackup(b *testing.B) { func BenchmarkDatabaseFullBackup(b *testing.B) { const numStatements = 100000 - _, sqlDB, dir, cleanupFn := backupccl.BackupRestoreTestSetup(b, backupccl.MultiNode, - 0 /* numAccounts */, backupccl.InitManualReplication) + _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(b, multiNode, + 0 /* numAccounts */, InitManualReplication) defer cleanupFn() - restoreURI := backupccl.LocalFoo + "/restore" - fullURI := backupccl.LocalFoo + "/full" + restoreURI := localFoo + "/restore" + fullURI := localFoo + "/full" bankData := bank.FromRows(numStatements).Tables()[0] _, err := sampledataccl.ToBackup(b, bankData, dir, "foo/restore") @@ -139,7 +138,7 @@ func BenchmarkDatabaseFullBackup(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - backupDir := backupccl.LocalFoo + fmt.Sprintf("/backup%d", i) + backupDir := localFoo + fmt.Sprintf("/backup%d", i) sqlDB.Exec(b, `BACKUP DATABASE data TO $1`, backupDir) } b.StopTimer() diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 53e53802d9b8..1e6d52e4a88e 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -161,11 +161,11 @@ CREATE TABLE data2.foo (a int); // Populate system.scheduled_jobs table with a first run in the future to prevent immediate adoption. firstRun := timeutil.Now().Add(time.Hour).Format(timeutil.TimestampWithoutTZFormat) - sqlDB.Exec(t, `CREATE SCHEDULE FOR BACKUP data.bank INTO $1 RECURRING '@hourly' FULL BACKUP ALWAYS WITH SCHEDULE OPTIONS first_run = $2`, LocalFoo, firstRun) + sqlDB.Exec(t, `CREATE SCHEDULE FOR BACKUP data.bank INTO $1 RECURRING '@hourly' FULL BACKUP ALWAYS WITH SCHEDULE OPTIONS first_run = $2`, localFoo, firstRun) sqlDB.Exec(t, `PAUSE SCHEDULES SELECT id FROM [SHOW SCHEDULES FOR BACKUP]`) injectStats(t, sqlDB, "data.bank", "id") - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) // Create a bunch of user tables on the restoring cluster that we're going // to delete. @@ -187,7 +187,7 @@ CREATE TABLE data2.foo (a int); doneRestore := make(chan struct{}) go func() { - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) close(doneRestore) }() @@ -374,16 +374,16 @@ func TestIncrementalFullClusterBackup(t *testing.T) { const numAccounts = 10 const incrementalBackupLocation = "nodelocal://0/inc-full-backup" - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() defer cleanupEmptyCluster() - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) sqlDB.Exec(t, "CREATE USER maxroach1") - sqlDB.Exec(t, `BACKUP TO $1 INCREMENTAL FROM $2`, incrementalBackupLocation, LocalFoo) - sqlDBRestore.Exec(t, `RESTORE FROM $1, $2`, LocalFoo, incrementalBackupLocation) + sqlDB.Exec(t, `BACKUP TO $1 INCREMENTAL FROM $2`, incrementalBackupLocation, localFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1, $2`, localFoo, incrementalBackupLocation) checkQuery := "SELECT * FROM system.users" sqlDBRestore.CheckQueryResults(t, checkQuery, sqlDB.QueryStr(t, checkQuery)) @@ -402,8 +402,8 @@ func TestEmptyFullClusterRestore(t *testing.T) { sqlDB.Exec(t, `CREATE USER alice`) sqlDB.Exec(t, `CREATE USER bob`) - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) checkQuery := "SELECT * FROM system.users" sqlDBRestore.CheckQueryResults(t, checkQuery, sqlDB.QueryStr(t, checkQuery)) @@ -415,15 +415,15 @@ func TestClusterRestoreEmptyDB(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() defer cleanupEmptyCluster() sqlDB.Exec(t, `CREATE DATABASE some_db`) sqlDB.Exec(t, `CREATE DATABASE some_db_2`) - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) checkQuery := "SHOW DATABASES" sqlDBRestore.CheckQueryResults(t, checkQuery, sqlDB.QueryStr(t, checkQuery)) @@ -434,16 +434,16 @@ func TestDisallowFullClusterRestoreOnNonFreshCluster(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() defer cleanupEmptyCluster() - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) sqlDBRestore.Exec(t, `CREATE DATABASE foo`) sqlDBRestore.ExpectErr(t, "pq: full cluster restore can only be run on a cluster with no tables or databases but found 2 descriptors: foo, public", - `RESTORE FROM $1`, LocalFoo, + `RESTORE FROM $1`, localFoo, ) } @@ -452,7 +452,7 @@ func TestClusterRestoreSystemTableOrdering(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) tcRestore, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) @@ -474,8 +474,8 @@ func TestClusterRestoreSystemTableOrdering(t *testing.T) { } } - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) // Check that the settings table is the last of the system tables to be // restored. require.Equal(t, restoredSystemTables[len(restoredSystemTables)-1], @@ -487,15 +487,15 @@ func TestDisallowFullClusterRestoreOfNonFullBackup(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() defer cleanupEmptyCluster() - sqlDB.Exec(t, `BACKUP data.bank TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank TO $1`, localFoo) sqlDBRestore.ExpectErr( t, "pq: full cluster RESTORE can only be used on full cluster BACKUP files", - `RESTORE FROM $1`, LocalFoo, + `RESTORE FROM $1`, localFoo, ) } @@ -504,12 +504,12 @@ func TestAllowNonFullClusterRestoreOfFullBackup(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) sqlDB.Exec(t, `CREATE DATABASE data2`) - sqlDB.Exec(t, `RESTORE data.bank FROM $1 WITH into_db='data2'`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.bank FROM $1 WITH into_db='data2'`, localFoo) checkResults := "SELECT * FROM data.bank" sqlDB.CheckQueryResults(t, checkResults, sqlDB.QueryStr(t, checkResults)) @@ -520,14 +520,14 @@ func TestRestoreFromFullClusterBackup(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 10 - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) sqlDB.Exec(t, `DROP DATABASE data`) t.Run("database", func(t *testing.T) { - sqlDB.Exec(t, `RESTORE DATABASE data FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE DATABASE data FROM $1`, localFoo) defer sqlDB.Exec(t, `DROP DATABASE data`) sqlDB.CheckQueryResults(t, "SELECT count(*) FROM data.bank", [][]string{{"10"}}) }) @@ -535,20 +535,20 @@ func TestRestoreFromFullClusterBackup(t *testing.T) { t.Run("table", func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE data`) defer sqlDB.Exec(t, `DROP DATABASE data`) - sqlDB.Exec(t, `RESTORE data.bank FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.bank FROM $1`, localFoo) sqlDB.CheckQueryResults(t, "SELECT count(*) FROM data.bank", [][]string{{"10"}}) }) t.Run("tables", func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE data`) defer sqlDB.Exec(t, `DROP DATABASE data`) - sqlDB.Exec(t, `RESTORE data.* FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE data.* FROM $1`, localFoo) sqlDB.CheckQueryResults(t, "SELECT count(*) FROM data.bank", [][]string{{"10"}}) }) t.Run("system tables", func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE temp_sys`) - sqlDB.Exec(t, `RESTORE system.users FROM $1 WITH into_db='temp_sys'`, LocalFoo) + sqlDB.Exec(t, `RESTORE system.users FROM $1 WITH into_db='temp_sys'`, localFoo) sqlDB.CheckQueryResults(t, "SELECT * FROM temp_sys.users", sqlDB.QueryStr(t, "SELECT * FROM system.users")) }) } @@ -557,15 +557,15 @@ func TestCreateDBAndTableIncrementalFullClusterBackup(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, 0, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) sqlDB.Exec(t, `CREATE DATABASE foo`) sqlDB.Exec(t, `CREATE TABLE foo.bar (a int)`) // Ensure that the new backup succeeds. - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) } // TestClusterRestoreFailCleanup tests that a failed RESTORE is cleaned up. @@ -585,7 +585,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } const numAccounts = 1000 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() // Setup the system systemTablesToVerify to ensure that they are copied to the new cluster. @@ -616,7 +616,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { // should appear in the restore. // This job will eventually fail since it will run from a new cluster. sqlDB.Exec(t, `BACKUP data.bank TO 'nodelocal://0/throwawayjob'`) - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) t.Run("during restoration of data", func(t *testing.T) { _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) @@ -682,7 +682,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } } // The initial restore will return an error, and restart. - sqlDBRestore.ExpectErr(t, `running execution from '.*' to '.*' on \d+ failed: injected error`, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.ExpectErr(t, `running execution from '.*' to '.*' on \d+ failed: injected error`, `RESTORE FROM $1`, localFoo) // Reduce retry delays. sqlDBRestore.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.initial_delay = '1ms'") // Expect the restore to succeed. @@ -711,7 +711,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } } - sqlDBRestore.ExpectErr(t, "injected error", `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.ExpectErr(t, "injected error", `RESTORE FROM $1`, localFoo) // Verify the failed RESTORE added some DROP tables. // Note that the system tables here correspond to the temporary tables // imported, not the system tables themselves. @@ -752,7 +752,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } } - sqlDBRestore.ExpectErr(t, "injected error", `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.ExpectErr(t, "injected error", `RESTORE FROM $1`, localFoo) }) } @@ -763,20 +763,20 @@ func TestDropDatabaseRevisionHistory(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, localFoo) sqlDB.Exec(t, ` CREATE DATABASE same_name_db; DROP DATABASE same_name_db; CREATE DATABASE same_name_db; `) - sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, localFoo) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupEmptyCluster() - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) sqlDBRestore.ExpectErr(t, `database "same_name_db" already exists`, `CREATE DATABASE same_name_db`) } @@ -796,7 +796,7 @@ func TestClusterRevisionHistory(t *testing.T) { var tc testCase const numAccounts = 1 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() sqlDB.Exec(t, `CREATE DATABASE d1`) sqlDB.Exec(t, `CREATE TABLE d1.t (a INT)`) @@ -835,7 +835,7 @@ func TestClusterRevisionHistory(t *testing.T) { }, } testCases = append(testCases, tc) - sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, localFoo) sqlDB.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&ts[3]) sqlDB.Exec(t, `DROP DATABASE d2;`) @@ -849,7 +849,7 @@ func TestClusterRevisionHistory(t *testing.T) { }, } testCases = append(testCases, tc) - sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, localFoo) sqlDB.Exec(t, `CREATE DATABASE d1`) sqlDB.Exec(t, `CREATE TABLE d1.t (a INT)`) @@ -877,14 +877,14 @@ func TestClusterRevisionHistory(t *testing.T) { }, } testCases = append(testCases, tc) - sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1 WITH revision_history`, localFoo) for i, testCase := range testCases { t.Run(fmt.Sprintf("t%d", i), func(t *testing.T) { _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupEmptyCluster() - sqlDBRestore.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+testCase.ts, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+testCase.ts, localFoo) testCase.check(t, sqlDBRestore) }) } @@ -1035,7 +1035,7 @@ func TestClusterRevisionDoesNotBackupOptOutSystemTables(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc, _, _, cleanup := BackupRestoreTestSetup(t, singleNode, 10, InitManualReplication) + tc, _, _, cleanup := backupRestoreTestSetup(t, singleNode, 10, InitManualReplication) conn := tc.Conns[0] sqlDB := sqlutils.MakeSQLRunner(conn) defer cleanup() @@ -1067,9 +1067,9 @@ CREATE DATABASE defaultdb; row := sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'defaultdb'`) var expectedDefaultDBID string row.Scan(&expectedDefaultDBID) - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) sqlDBRestore.CheckQueryResults(t, `SELECT * FROM system.namespace WHERE name = 'defaultdb'`, [][]string{ {"0", "0", "defaultdb", expectedDefaultDBID}, @@ -1088,9 +1088,9 @@ func TestRestoreWithDroppedDefaultDB(t *testing.T) { sqlDB.Exec(t, ` DROP DATABASE defaultdb; `) - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) sqlDBRestore.CheckQueryResults(t, `SELECT count(*) FROM system.namespace WHERE name = 'defaultdb'`, [][]string{ {"0"}, @@ -1111,12 +1111,12 @@ func TestRestoreToClusterWithDroppedDefaultDB(t *testing.T) { var name string expectedRow.Scan(&parentID, &parentSchemaID, &name, &ID) - sqlDB.Exec(t, `BACKUP TO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP TO $1`, localFoo) sqlDBRestore.Exec(t, ` DROP DATABASE defaultdb; `) - sqlDBRestore.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDBRestore.Exec(t, `RESTORE FROM $1`, localFoo) sqlDBRestore.CheckQueryResults(t, `SELECT * FROM system.namespace WHERE name = 'defaultdb'`, [][]string{ {fmt.Sprint(parentID), fmt.Sprint(parentSchemaID), name, fmt.Sprint(ID)}, }) diff --git a/pkg/ccl/backupccl/helpers_test.go b/pkg/ccl/backupccl/helpers_test.go deleted file mode 100644 index b8cadf06c715..000000000000 --- a/pkg/ccl/backupccl/helpers_test.go +++ /dev/null @@ -1,398 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Licensed as a CockroachDB Enterprise file under the Cockroach Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt - -package backupccl - -import ( - "context" - gosql "database/sql" - "fmt" - "io" - "io/ioutil" - "net/http" - "net/http/httptest" - "net/url" - "os" - "path/filepath" - "reflect" - "strings" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" - "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/cockroach/pkg/workload/bank" - "github.com/cockroachdb/cockroach/pkg/workload/workloadsql" - "github.com/cockroachdb/errors" - "github.com/cockroachdb/logtags" - "github.com/kr/pretty" - "github.com/stretchr/testify/require" -) - -const ( - singleNode = 1 - MultiNode = 3 - backupRestoreDefaultRanges = 10 - backupRestoreRowPayloadSize = 100 - LocalFoo = "nodelocal://0/foo" -) - -func backupRestoreTestSetupWithParams( - t testing.TB, - clusterSize int, - numAccounts int, - init func(tc *testcluster.TestCluster), - params base.TestClusterArgs, -) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { - ctx := logtags.AddTag(context.Background(), "backup-restore-test-setup", nil) - - dir, dirCleanupFn := testutils.TempDir(t) - params.ServerArgs.ExternalIODir = dir - params.ServerArgs.UseDatabase = "data" - if len(params.ServerArgsPerNode) > 0 { - for i := range params.ServerArgsPerNode { - param := params.ServerArgsPerNode[i] - param.ExternalIODir = dir - param.UseDatabase = "data" - params.ServerArgsPerNode[i] = param - } - } - - tc = testcluster.StartTestCluster(t, clusterSize, params) - init(tc) - - const payloadSize = 100 - splits := 10 - if numAccounts == 0 { - splits = 0 - } - bankData := bank.FromConfig(numAccounts, numAccounts, payloadSize, splits) - - sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) - - // Set the max buffer size to something low to prevent backup/restore tests - // from hitting OOM errors. If any test cares about this setting in - // particular, they will override it inline after setting up the test cluster. - sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.merge_file_buffer_size = '16MiB'`) - - sqlDB.Exec(t, `CREATE DATABASE data`) - l := workloadsql.InsertsDataLoader{BatchSize: 1000, Concurrency: 4} - if _, err := workloadsql.Setup(ctx, sqlDB.DB.(*gosql.DB), bankData, l); err != nil { - t.Fatalf("%+v", err) - } - - if err := tc.WaitForFullReplication(); err != nil { - t.Fatal(err) - } - - cleanupFn := func() { - tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs - dirCleanupFn() // cleans up dir, which is the nodelocal:// storage - } - - return tc, sqlDB, dir, cleanupFn -} - -func BackupRestoreTestSetup( - t testing.TB, clusterSize int, numAccounts int, init func(*testcluster.TestCluster), -) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { - return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init, base.TestClusterArgs{}) -} - -func backupRestoreTestSetupEmpty( - t testing.TB, - clusterSize int, - tempDir string, - init func(*testcluster.TestCluster), - params base.TestClusterArgs, -) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, cleanup func()) { - return backupRestoreTestSetupEmptyWithParams(t, clusterSize, tempDir, init, params) -} - -func verifyBackupRestoreStatementResult( - t *testing.T, sqlDB *sqlutils.SQLRunner, query string, args ...interface{}, -) error { - t.Helper() - rows := sqlDB.Query(t, query, args...) - - columns, err := rows.Columns() - if err != nil { - return err - } - if e, a := columns, []string{ - "job_id", "status", "fraction_completed", "rows", "index_entries", "bytes", - }; !reflect.DeepEqual(e, a) { - return errors.Errorf("unexpected columns:\n%s", strings.Join(pretty.Diff(e, a), "\n")) - } - - type job struct { - id int64 - status string - fractionCompleted float32 - } - - var expectedJob job - var actualJob job - var unused int64 - - if !rows.Next() { - if err := rows.Err(); err != nil { - return err - } - return errors.New("zero rows in result") - } - if err := rows.Scan( - &actualJob.id, &actualJob.status, &actualJob.fractionCompleted, &unused, &unused, &unused, - ); err != nil { - return err - } - if rows.Next() { - return errors.New("more than one row in result") - } - - sqlDB.QueryRow(t, - `SELECT job_id, status, fraction_completed FROM crdb_internal.jobs WHERE job_id = $1`, actualJob.id, - ).Scan( - &expectedJob.id, &expectedJob.status, &expectedJob.fractionCompleted, - ) - - if e, a := expectedJob, actualJob; !reflect.DeepEqual(e, a) { - return errors.Errorf("result does not match system.jobs:\n%s", - strings.Join(pretty.Diff(e, a), "\n")) - } - - return nil -} - -func backupRestoreTestSetupEmptyWithParams( - t testing.TB, - clusterSize int, - dir string, - init func(tc *testcluster.TestCluster), - params base.TestClusterArgs, -) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, cleanup func()) { - ctx := logtags.AddTag(context.Background(), "backup-restore-test-setup-empty", nil) - - params.ServerArgs.ExternalIODir = dir - if len(params.ServerArgsPerNode) > 0 { - for i := range params.ServerArgsPerNode { - param := params.ServerArgsPerNode[i] - param.ExternalIODir = dir - params.ServerArgsPerNode[i] = param - } - } - tc = testcluster.StartTestCluster(t, clusterSize, params) - init(tc) - - sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) - - cleanupFn := func() { - tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs - } - - return tc, sqlDB, cleanupFn -} - -func createEmptyCluster( - t testing.TB, clusterSize int, -) (sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { - ctx := context.Background() - - dir, dirCleanupFn := testutils.TempDir(t) - params := base.TestClusterArgs{} - params.ServerArgs.ExternalIODir = dir - tc := testcluster.StartTestCluster(t, clusterSize, params) - - sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) - - cleanupFn := func() { - tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs - dirCleanupFn() // cleans up dir, which is the nodelocal:// storage - } - - return sqlDB, dir, cleanupFn -} - -// getStatsQuery returns a SQL query that will return the properties of the -// statistics on a table that are expected to remain the same after being -// restored on a new cluster. -func getStatsQuery(tableName string) string { - return fmt.Sprintf(`SELECT - statistics_name, - column_names, - row_count, - distinct_count, - null_count - FROM [SHOW STATISTICS FOR TABLE %s]`, tableName) -} - -// injectStats directly injects some arbitrary statistic into a given table for -// a specified column. -// See injectStatsWithRowCount. -func injectStats( - t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, columnName string, -) [][]string { - return injectStatsWithRowCount(t, sqlDB, tableName, columnName, 100 /* rowCount */) -} - -// injectStatsWithRowCount directly injects some statistics specifying some row -// count for a column in the given table. -// N.B. This should be used in backup testing over CREATE STATISTICS since it -// ensures that the stats cache will be up to date during a subsequent BACKUP. -func injectStatsWithRowCount( - t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, columnName string, rowCount int, -) [][]string { - sqlDB.Exec(t, fmt.Sprintf(`ALTER TABLE %s INJECT STATISTICS '[ - { - "columns": ["%s"], - "created_at": "2018-01-01 1:00:00.00000+00:00", - "row_count": %d, - "distinct_count": %d - } - ]'`, tableName, columnName, rowCount, rowCount)) - return sqlDB.QueryStr(t, getStatsQuery(tableName)) -} - -func makeInsecureHTTPServer(t *testing.T) (string, func()) { - t.Helper() - - const badHeadResponse = "bad-head-response" - - tmp, dirCleanup := testutils.TempDir(t) - srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - localfile := filepath.Join(tmp, filepath.Base(r.URL.Path)) - switch r.Method { - case "PUT": - f, err := os.Create(localfile) - if err != nil { - http.Error(w, err.Error(), 500) - return - } - defer f.Close() - if _, err := io.Copy(f, r.Body); err != nil { - http.Error(w, err.Error(), 500) - return - } - w.WriteHeader(201) - case "GET", "HEAD": - if filepath.Base(localfile) == badHeadResponse { - http.Error(w, "HEAD not implemented", 500) - return - } - http.ServeFile(w, r, localfile) - case "DELETE": - if err := os.Remove(localfile); err != nil { - http.Error(w, err.Error(), 500) - return - } - w.WriteHeader(204) - default: - http.Error(w, "unsupported method "+r.Method, 400) - } - })) - - cleanup := func() { - srv.Close() - dirCleanup() - } - - t.Logf("Mock HTTP Storage %q", srv.URL) - uri, err := url.Parse(srv.URL) - if err != nil { - srv.Close() - t.Fatal(err) - } - uri.Path = filepath.Join(uri.Path, "testing") - return uri.String(), cleanup -} - -// thresholdBlocker is a small wrapper around channels that are commonly used to -// block operations during testing. -// For example, it can be used in conjection with the RunBeforeBackfillChunk and -// BulkAdderFlushesEveryBatch cluster settings. The SQLSchemaChanger knob can be -// used to control the chunk size. -type thresholdBlocker struct { - threshold int - reachedThreshold chan struct{} - canProceed chan struct{} -} - -func (t thresholdBlocker) maybeBlock(count int) { - if count == t.threshold { - close(t.reachedThreshold) - <-t.canProceed - } -} - -func (t thresholdBlocker) waitUntilBlocked() { - <-t.reachedThreshold -} - -func (t thresholdBlocker) allowToProceed() { - close(t.canProceed) -} - -func makeThresholdBlocker(threshold int) thresholdBlocker { - return thresholdBlocker{ - threshold: threshold, - reachedThreshold: make(chan struct{}), - canProceed: make(chan struct{}), - } -} - -// getSpansFromManifest returns the spans that describe the data included in a -// given backup. -func getSpansFromManifest(ctx context.Context, t *testing.T, backupPath string) roachpb.Spans { - backupManifestBytes, err := ioutil.ReadFile(backupPath + "/" + backupManifestName) - require.NoError(t, err) - var backupManifest BackupManifest - decompressedBytes, err := decompressData(ctx, nil, backupManifestBytes) - require.NoError(t, err) - require.NoError(t, protoutil.Unmarshal(decompressedBytes, &backupManifest)) - spans := make([]roachpb.Span, 0, len(backupManifest.Files)) - for _, file := range backupManifest.Files { - spans = append(spans, file.Span) - } - mergedSpans, _ := roachpb.MergeSpans(&spans) - return mergedSpans -} - -func getKVCount(ctx context.Context, kvDB *kv.DB, dbName, tableName string) (int, error) { - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, tableName) - tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.GetID())) - tableEnd := tablePrefix.PrefixEnd() - kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0) - return len(kvs), err -} - -// uriFmtStringAndArgs returns format strings like "$1" or "($1, $2, $3)" and -// an []interface{} of URIs for the BACKUP/RESTORE queries. -func uriFmtStringAndArgs(uris []string) (string, []interface{}) { - urisForFormat := make([]interface{}, len(uris)) - var fmtString strings.Builder - if len(uris) > 1 { - fmtString.WriteString("(") - } - for i, uri := range uris { - if i > 0 { - fmtString.WriteString(", ") - } - fmtString.WriteString(fmt.Sprintf("$%d", i+1)) - urisForFormat[i] = uri - } - if len(uris) > 1 { - fmtString.WriteString(")") - } - return fmtString.String(), urisForFormat -} diff --git a/pkg/ccl/backupccl/restore_mid_schema_change_test.go b/pkg/ccl/backupccl/restore_mid_schema_change_test.go index 4d94fdfcccc5..89da8db3eadc 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -255,8 +255,8 @@ func restoreMidSchemaChange( if isClusterRestore { restoreQuery = "RESTORE from $1" } - log.Infof(context.Background(), "%+v", sqlDB.QueryStr(t, "SHOW BACKUP $1", LocalFoo)) - sqlDB.Exec(t, restoreQuery, LocalFoo) + log.Infof(context.Background(), "%+v", sqlDB.QueryStr(t, "SHOW BACKUP $1", localFoo)) + sqlDB.Exec(t, restoreQuery, localFoo) // Wait for all jobs to terminate. Some may fail since we don't restore // adding spans. sqlDB.CheckQueryResultsRetry(t, "SELECT * FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND NOT (status = 'succeeded' OR status = 'failed')", [][]string{}) diff --git a/pkg/ccl/backupccl/restore_old_sequences_test.go b/pkg/ccl/backupccl/restore_old_sequences_test.go index 56222c79ca78..4934b4cdf28f 100644 --- a/pkg/ccl/backupccl/restore_old_sequences_test.go +++ b/pkg/ccl/backupccl/restore_old_sequences_test.go @@ -71,7 +71,7 @@ func restoreOldSequencesTest(exportDir string) func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE test`) var unused string var importedRows int - sqlDB.QueryRow(t, `RESTORE test.* FROM $1`, LocalFoo).Scan( + sqlDB.QueryRow(t, `RESTORE test.* FROM $1`, localFoo).Scan( &unused, &unused, &unused, &importedRows, &unused, &unused, ) const totalRows = 4 diff --git a/pkg/ccl/backupccl/restore_old_versions_test.go b/pkg/ccl/backupccl/restore_old_versions_test.go index fc2580bdc03c..f9d20a2b9d37 100644 --- a/pkg/ccl/backupccl/restore_old_versions_test.go +++ b/pkg/ccl/backupccl/restore_old_versions_test.go @@ -170,9 +170,9 @@ func TestRestoreOldVersions(t *testing.T) { err = os.Symlink(exportDir, filepath.Join(externalDir, "foo")) require.NoError(t, err) - sqlDB.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE FROM $1`, localFoo) sqlDB.Exec(t, `DROP DATABASE db1;`) - sqlDB.Exec(t, `RESTORE DATABASE db1 FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE DATABASE db1 FROM $1`, localFoo) sqlDB.CheckQueryResults(t, `SELECT count(*) FROM [SHOW DATABASES] WHERE database_name = 'db1'`, [][]string{{"1"}}, @@ -217,14 +217,14 @@ func TestRestoreOldVersions(t *testing.T) { require.NoError(t, err) // Expect this restore to fail. - sqlDB.ExpectErr(t, `type "t" has unknown ParentID 50`, `RESTORE DATABASE otherdb FROM $1`, LocalFoo) + sqlDB.ExpectErr(t, `type "t" has unknown ParentID 50`, `RESTORE DATABASE otherdb FROM $1`, localFoo) // Expect that we don't crash and that we emit NULL for data that we // cannot resolve (e.g. missing database descriptor, create_statement). sqlDB.CheckQueryResults(t, ` SELECT database_name, parent_schema_name, object_name, object_type, create_statement -FROM [SHOW BACKUP SCHEMAS '`+LocalFoo+`' WITH privileges] +FROM [SHOW BACKUP SCHEMAS '`+localFoo+`' WITH privileges] ORDER BY object_type, object_name`, [][]string{ {"NULL", "NULL", "otherdb", "database", "NULL"}, {"otherdb", "public", "tbl", "table", "NULL"}, @@ -303,7 +303,7 @@ func restoreOldVersionTestWithInterleave(exportDir string) func(t *testing.T) { // Restore should now fail. sqlDB.ExpectErr(t, "pq: restoring interleaved tables is no longer allowed. table t3 was found to be interleaved", - `RESTORE test.* FROM $1`, LocalFoo) + `RESTORE test.* FROM $1`, localFoo) } } @@ -399,7 +399,7 @@ func restoreOldVersionTest(exportDir string) func(t *testing.T) { sqlDB.Exec(t, `CREATE DATABASE test`) var unused string var importedRows int - sqlDB.QueryRow(t, `RESTORE test.* FROM $1`, LocalFoo).Scan( + sqlDB.QueryRow(t, `RESTORE test.* FROM $1`, localFoo).Scan( &unused, &unused, &unused, &importedRows, &unused, &unused, ) const totalRows = 12 @@ -429,7 +429,7 @@ func restoreOldVersionTest(exportDir string) func(t *testing.T) { func restoreV201ZoneconfigPrivilegeTest(exportDir string) func(t *testing.T) { return func(t *testing.T) { const numAccounts = 1000 - _, _, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() _, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, @@ -437,7 +437,7 @@ func restoreV201ZoneconfigPrivilegeTest(exportDir string) func(t *testing.T) { defer cleanup() err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) require.NoError(t, err) - sqlDB.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE FROM $1`, localFoo) testDBGrants := [][]string{ {"test", "admin", "ALL", "true"}, {"test", "root", "ALL", "true"}, @@ -471,25 +471,25 @@ func restoreOldVersionFKRevTest(exportDir string) func(t *testing.T) { err := os.Symlink(exportDir, filepath.Join(dir, "foo")) require.NoError(t, err) sqlDB.Exec(t, `CREATE DATABASE ts`) - sqlDB.Exec(t, `RESTORE test.rev_times FROM $1 WITH into_db = 'ts'`, LocalFoo) + sqlDB.Exec(t, `RESTORE test.rev_times FROM $1 WITH into_db = 'ts'`, localFoo) for _, ts := range sqlDB.QueryStr(t, `SELECT logical_time FROM ts.rev_times`) { - sqlDB.Exec(t, fmt.Sprintf(`RESTORE DATABASE test FROM $1 AS OF SYSTEM TIME %s`, ts[0]), LocalFoo) + sqlDB.Exec(t, fmt.Sprintf(`RESTORE DATABASE test FROM $1 AS OF SYSTEM TIME %s`, ts[0]), localFoo) // Just rendering the constraints loads and validates schema. sqlDB.Exec(t, `SELECT * FROM pg_catalog.pg_constraint`) sqlDB.Exec(t, `DROP DATABASE test`) // Restore a couple tables, including parent but not child_pk. sqlDB.Exec(t, `CREATE DATABASE test`) - sqlDB.Exec(t, fmt.Sprintf(`RESTORE test.circular FROM $1 AS OF SYSTEM TIME %s`, ts[0]), LocalFoo) - sqlDB.Exec(t, fmt.Sprintf(`RESTORE test.parent, test.child FROM $1 AS OF SYSTEM TIME %s WITH skip_missing_foreign_keys`, ts[0]), LocalFoo) + sqlDB.Exec(t, fmt.Sprintf(`RESTORE test.circular FROM $1 AS OF SYSTEM TIME %s`, ts[0]), localFoo) + sqlDB.Exec(t, fmt.Sprintf(`RESTORE test.parent, test.child FROM $1 AS OF SYSTEM TIME %s WITH skip_missing_foreign_keys`, ts[0]), localFoo) sqlDB.Exec(t, `SELECT * FROM pg_catalog.pg_constraint`) sqlDB.Exec(t, `DROP DATABASE test`) // Now do each table on its own with skip_missing_foreign_keys. sqlDB.Exec(t, `CREATE DATABASE test`) for _, name := range []string{"child_pk", "child", "circular", "parent"} { - sqlDB.Exec(t, fmt.Sprintf(`RESTORE test.%s FROM $1 AS OF SYSTEM TIME %s WITH skip_missing_foreign_keys`, name, ts[0]), LocalFoo) + sqlDB.Exec(t, fmt.Sprintf(`RESTORE test.%s FROM $1 AS OF SYSTEM TIME %s WITH skip_missing_foreign_keys`, name, ts[0]), localFoo) } sqlDB.Exec(t, `SELECT * FROM pg_catalog.pg_constraint`) sqlDB.Exec(t, `DROP DATABASE test`) @@ -516,7 +516,7 @@ func restoreOldVersionClusterTest(exportDir string) func(t *testing.T) { require.NoError(t, err) // Ensure that the restore succeeds. - sqlDB.Exec(t, `RESTORE FROM $1`, LocalFoo) + sqlDB.Exec(t, `RESTORE FROM $1`, localFoo) sqlDB.CheckQueryResults(t, "SHOW USERS", [][]string{ {"admin", "", "{}"}, @@ -802,7 +802,7 @@ func TestRestoreWithDroppedSchemaCorruption(t *testing.T) { func restorePublicSchemaRemap(exportDir string) func(t *testing.T) { return func(t *testing.T) { const numAccounts = 1000 - _, _, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() _, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, @@ -811,7 +811,7 @@ func restorePublicSchemaRemap(exportDir string) func(t *testing.T) { err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) require.NoError(t, err) - sqlDB.Exec(t, fmt.Sprintf("RESTORE DATABASE d FROM '%s'", LocalFoo)) + sqlDB.Exec(t, fmt.Sprintf("RESTORE DATABASE d FROM '%s'", localFoo)) var restoredDBID, publicSchemaID int row := sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name='d' AND "parentID"=0`) @@ -834,7 +834,7 @@ func restorePublicSchemaRemap(exportDir string) func(t *testing.T) { // previously had a synthetic public schema gets correctly restored into the // descriptor backed public schema of database test. sqlDB.Exec(t, `CREATE DATABASE test`) - sqlDB.Exec(t, `RESTORE d.public.t FROM $1 WITH into_db = 'test'`, LocalFoo) + sqlDB.Exec(t, `RESTORE d.public.t FROM $1 WITH into_db = 'test'`, localFoo) row = sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name='test' AND "parentID"=0`) var parentDBID int @@ -856,7 +856,7 @@ func restorePublicSchemaRemap(exportDir string) func(t *testing.T) { func restorePublicSchemaMixedVersion(exportDir string) func(t *testing.T) { return func(t *testing.T) { const numAccounts = 1000 - _, _, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() _, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, @@ -874,7 +874,7 @@ func restorePublicSchemaMixedVersion(exportDir string) func(t *testing.T) { err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) require.NoError(t, err) - sqlDB.Exec(t, fmt.Sprintf("RESTORE DATABASE d FROM '%s'", LocalFoo)) + sqlDB.Exec(t, fmt.Sprintf("RESTORE DATABASE d FROM '%s'", localFoo)) var restoredDBID int row := sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name='d' AND "parentID"=0`) @@ -893,7 +893,7 @@ func restorePublicSchemaMixedVersion(exportDir string) func(t *testing.T) { // previously had a synthetic public schema gets correctly restored into the // descriptor backed public schema of database test. sqlDB.Exec(t, `CREATE DATABASE test`) - sqlDB.Exec(t, `RESTORE d.public.t FROM $1 WITH into_db = 'test'`, LocalFoo) + sqlDB.Exec(t, `RESTORE d.public.t FROM $1 WITH into_db = 'test'`, localFoo) row = sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name='test' AND "parentID"=0`) var parentDBID int @@ -911,7 +911,7 @@ func restorePublicSchemaMixedVersion(exportDir string) func(t *testing.T) { func restoreSyntheticPublicSchemaNamespaceEntry(exportDir string) func(t *testing.T) { return func(t *testing.T) { const numAccounts = 1000 - _, _, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() _, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, @@ -929,7 +929,7 @@ func restoreSyntheticPublicSchemaNamespaceEntry(exportDir string) func(t *testin err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) require.NoError(t, err) - sqlDB.Exec(t, fmt.Sprintf("RESTORE DATABASE d FROM '%s'", LocalFoo)) + sqlDB.Exec(t, fmt.Sprintf("RESTORE DATABASE d FROM '%s'", localFoo)) var dbID int row := sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'd'`) @@ -942,7 +942,7 @@ func restoreSyntheticPublicSchemaNamespaceEntry(exportDir string) func(t *testin func restoreSyntheticPublicSchemaNamespaceEntryCleanupOnFail(exportDir string) func(t *testing.T) { return func(t *testing.T) { const numAccounts = 1000 - _, _, tmpDir, cleanupFn := BackupRestoreTestSetup(t, MultiNode, numAccounts, InitManualReplication) + _, _, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) defer cleanupFn() tc, sqlDB, cleanup := backupRestoreTestSetupEmpty(t, singleNode, tmpDir, @@ -977,7 +977,7 @@ func restoreSyntheticPublicSchemaNamespaceEntryCleanupOnFail(exportDir string) f sqlDB.Exec(t, "DROP DATABASE defaultdb") sqlDB.Exec(t, "DROP DATABASE postgres") - restoreQuery := fmt.Sprintf("RESTORE DATABASE d FROM '%s'", LocalFoo) + restoreQuery := fmt.Sprintf("RESTORE DATABASE d FROM '%s'", localFoo) sqlDB.ExpectErr(t, "boom", restoreQuery) // We should have no non-system database with a public schema name space diff --git a/pkg/ccl/backupccl/show_test.go b/pkg/ccl/backupccl/show_test.go index 1c8ef9af5641..f1e2f17a38e8 100644 --- a/pkg/ccl/backupccl/show_test.go +++ b/pkg/ccl/backupccl/show_test.go @@ -41,7 +41,7 @@ func TestShowBackup(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 11 - tc, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) kvDB := tc.Server(0).DB() _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() @@ -54,7 +54,7 @@ CREATE TABLE data.sc.t1 (a INT); CREATE TABLE data.sc.t2 (a data.welcome); `) - const full, inc, inc2 = LocalFoo + "/full", LocalFoo + "/inc", LocalFoo + "/inc2" + const full, inc, inc2 = localFoo + "/full", localFoo + "/inc", localFoo + "/inc2" beforeTS := sqlDB.QueryStr(t, `SELECT now()::timestamp::string`)[0][0] sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s'`, beforeTS), full) @@ -157,7 +157,7 @@ ORDER BY object_type, object_name`, full) {"users", incTS, inc2TS, "3"}, }, res) - const details = LocalFoo + "/details" + const details = localFoo + "/details" sqlDB.Exec(t, `CREATE TABLE data.details1 (c INT PRIMARY KEY)`) sqlDB.Exec(t, `INSERT INTO data.details1 (SELECT generate_series(1, 100))`) sqlDB.Exec(t, `ALTER TABLE data.details1 SPLIT AT VALUES (1), (42)`) @@ -217,7 +217,7 @@ ORDER BY object_type, object_name`, full) // Test that tables, views and sequences are all supported. { - viewTableSeq := LocalFoo + "/tableviewseq" + viewTableSeq := localFoo + "/tableviewseq" sqlDB.Exec(t, `CREATE TABLE data.tableA (a int primary key, b int, INDEX tableA_b_idx (b ASC))`) sqlDB.Exec(t, `CREATE VIEW data.viewA AS SELECT a from data.tableA`) sqlDB.Exec(t, `CREATE SEQUENCE data.seqA START 1 INCREMENT 2 MAXVALUE 20`) @@ -259,7 +259,7 @@ ORDER BY object_type, object_name`, full) // Test that foreign keys that reference tables that are in the backup // are included. { - includedFK := LocalFoo + "/includedFK" + includedFK := localFoo + "/includedFK" sqlDB.Exec(t, `CREATE TABLE data.FKSrc (a INT PRIMARY KEY)`) sqlDB.Exec(t, `CREATE TABLE data.FKRefTable (a INT PRIMARY KEY, B INT REFERENCES data.FKSrc(a))`) sqlDB.Exec(t, `CREATE DATABASE data2`) @@ -294,7 +294,7 @@ ORDER BY object_type, object_name`, full) // Foreign keys that were not included in the backup are not mentioned in // the create statement. { - missingFK := LocalFoo + "/missingFK" + missingFK := localFoo + "/missingFK" sqlDB.Exec(t, `BACKUP data2.FKRefTable TO $1;`, missingFK) want := `CREATE TABLE fkreftable ( @@ -311,7 +311,7 @@ ORDER BY object_type, object_name`, full) } { - fullCluster := LocalFoo + "/full_cluster" + fullCluster := localFoo + "/full_cluster" sqlDB.Exec(t, `BACKUP TO $1;`, fullCluster) showBackupRows = sqlDBRestore.QueryStr(t, fmt.Sprintf(`SELECT is_full_cluster FROM [SHOW BACKUP '%s']`, fullCluster)) @@ -320,7 +320,7 @@ ORDER BY object_type, object_name`, full) t.Fatal("expected show backup to indicate that backup was full cluster") } - fullClusterInc := LocalFoo + "/full_cluster_inc" + fullClusterInc := localFoo + "/full_cluster_inc" sqlDB.Exec(t, `BACKUP TO $1 INCREMENTAL FROM $2;`, fullClusterInc, fullCluster) showBackupRows = sqlDBRestore.QueryStr(t, fmt.Sprintf(`SELECT is_full_cluster FROM [SHOW BACKUP '%s']`, fullCluster)) @@ -332,7 +332,7 @@ ORDER BY object_type, object_name`, full) // Show privileges of descriptors that are backed up. { - showPrivs := LocalFoo + "/show_privs" + showPrivs := localFoo + "/show_privs" sqlDB.Exec(t, ` CREATE DATABASE mi5; USE mi5; CREATE SCHEMA locator; @@ -390,7 +390,7 @@ GRANT UPDATE ON top_secret TO agent_bond; sqlDBRestore.CheckQueryResults(t, showQuery, want) // Change the owner and expect the changes to be reflected in a new backup - showOwner := LocalFoo + "/show_owner" + showOwner := localFoo + "/show_owner" sqlDB.Exec(t, ` ALTER DATABASE mi5 OWNER TO agent_thomas; ALTER SCHEMA locator OWNER TO agent_thomas; @@ -423,13 +423,13 @@ func TestShowBackups(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 11 - _, sqlDB, tempDir, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, tempDir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) _, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() defer cleanupEmptyCluster() - const full = LocalFoo + "/full" - const remoteInc = LocalFoo + "/inc" + const full = localFoo + "/full" + const remoteInc = localFoo + "/inc" // Make an initial backup. sqlDB.Exec(t, `BACKUP data.bank INTO $1`, full) @@ -476,7 +476,7 @@ func TestShowBackupTenants(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 1 - tc, systemDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, systemDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() srv := tc.Server(0) @@ -544,7 +544,7 @@ func TestShowBackupPrivileges(t *testing.T) { }() // Make an initial backup. - const full = LocalFoo + "/full" + const full = localFoo + "/full" sqlDB.Exec(t, `BACKUP privs INTO $1`, full) // Add an incremental backup to it. sqlDB.Exec(t, `BACKUP privs INTO LATEST IN $1`, full) @@ -621,7 +621,7 @@ func showUpgradedForeignKeysTest(exportDir string) func(t *testing.T) { [SHOW BACKUP SCHEMAS $1] WHERE object_type = 'table' AND object_name = $2 - `, LocalFoo, tc.table) + `, localFoo, tc.table) require.NotEmpty(t, results) require.Regexp(t, regexp.MustCompile(tc.expectedForeignKeyPattern), results[0][0]) } @@ -634,7 +634,7 @@ func TestShowBackupWithDebugIDs(t *testing.T) { const numAccounts = 11 // Create test database with bank table - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() // add 1 type, 1 schema, and 2 tables to the database @@ -646,7 +646,7 @@ func TestShowBackupWithDebugIDs(t *testing.T) { CREATE TABLE data.sc.t2 (a data.welcome); `) - const full = LocalFoo + "/full" + const full = localFoo + "/full" beforeTS := sqlDB.QueryStr(t, `SELECT now()::timestamp::string`)[0][0] sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s'`, beforeTS), full) @@ -696,13 +696,13 @@ func TestShowBackupPathIsCollectionRoot(t *testing.T) { const numAccounts = 11 // Create test database with bank table. - _, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() // Make an initial backup. - sqlDB.Exec(t, `BACKUP data.bank INTO $1`, LocalFoo) + sqlDB.Exec(t, `BACKUP data.bank INTO $1`, localFoo) // Ensure proper error gets returned from back SHOW BACKUP Path sqlDB.ExpectErr(t, "The specified path is the root of a backup collection.", - "SHOW BACKUP $1", LocalFoo) + "SHOW BACKUP $1", localFoo) } diff --git a/pkg/ccl/backupccl/testutils.go b/pkg/ccl/backupccl/testutils.go index fdb6b0f74add..53c250a5aa02 100644 --- a/pkg/ccl/backupccl/testutils.go +++ b/pkg/ccl/backupccl/testutils.go @@ -1,4 +1,4 @@ -// Copyright 2021 The Cockroach Authors. +// Copyright 2020 The Cockroach Authors. // // Licensed as a CockroachDB Enterprise file under the Cockroach Community // License (the "License"); you may not use this file except in compliance with @@ -9,19 +9,401 @@ package backupccl import ( + "context" + gosql "database/sql" "encoding/json" + "fmt" + "io" + "io/ioutil" "math" + "net/http" + "net/http/httptest" + "net/url" + "os" + "path/filepath" + "reflect" "regexp" "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/workload/bank" + "github.com/cockroachdb/cockroach/pkg/workload/workloadsql" "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" + "github.com/kr/pretty" "github.com/stretchr/testify/require" ) +const ( + singleNode = 1 + multiNode = 3 + backupRestoreDefaultRanges = 10 + backupRestoreRowPayloadSize = 100 + localFoo = "nodelocal://0/foo" +) + +func backupRestoreTestSetupWithParams( + t testing.TB, + clusterSize int, + numAccounts int, + init func(tc *testcluster.TestCluster), + params base.TestClusterArgs, +) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { + ctx := logtags.AddTag(context.Background(), "backup-restore-test-setup", nil) + + dir, dirCleanupFn := testutils.TempDir(t) + params.ServerArgs.ExternalIODir = dir + params.ServerArgs.UseDatabase = "data" + if len(params.ServerArgsPerNode) > 0 { + for i := range params.ServerArgsPerNode { + param := params.ServerArgsPerNode[i] + param.ExternalIODir = dir + param.UseDatabase = "data" + params.ServerArgsPerNode[i] = param + } + } + + tc = testcluster.StartTestCluster(t, clusterSize, params) + init(tc) + + const payloadSize = 100 + splits := 10 + if numAccounts == 0 { + splits = 0 + } + bankData := bank.FromConfig(numAccounts, numAccounts, payloadSize, splits) + + sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) + + // Set the max buffer size to something low to prevent backup/restore tests + // from hitting OOM errors. If any test cares about this setting in + // particular, they will override it inline after setting up the test cluster. + sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.merge_file_buffer_size = '16MiB'`) + + sqlDB.Exec(t, `CREATE DATABASE data`) + l := workloadsql.InsertsDataLoader{BatchSize: 1000, Concurrency: 4} + if _, err := workloadsql.Setup(ctx, sqlDB.DB.(*gosql.DB), bankData, l); err != nil { + t.Fatalf("%+v", err) + } + + if err := tc.WaitForFullReplication(); err != nil { + t.Fatal(err) + } + + cleanupFn := func() { + tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs + dirCleanupFn() // cleans up dir, which is the nodelocal:// storage + } + + return tc, sqlDB, dir, cleanupFn +} + +func backupRestoreTestSetup( + t testing.TB, clusterSize int, numAccounts int, init func(*testcluster.TestCluster), +) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { + return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init, base.TestClusterArgs{}) +} + +func backupRestoreTestSetupEmpty( + t testing.TB, + clusterSize int, + tempDir string, + init func(*testcluster.TestCluster), + params base.TestClusterArgs, +) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, cleanup func()) { + return backupRestoreTestSetupEmptyWithParams(t, clusterSize, tempDir, init, params) +} + +func verifyBackupRestoreStatementResult( + t *testing.T, sqlDB *sqlutils.SQLRunner, query string, args ...interface{}, +) error { + t.Helper() + rows := sqlDB.Query(t, query, args...) + + columns, err := rows.Columns() + if err != nil { + return err + } + if e, a := columns, []string{ + "job_id", "status", "fraction_completed", "rows", "index_entries", "bytes", + }; !reflect.DeepEqual(e, a) { + return errors.Errorf("unexpected columns:\n%s", strings.Join(pretty.Diff(e, a), "\n")) + } + + type job struct { + id int64 + status string + fractionCompleted float32 + } + + var expectedJob job + var actualJob job + var unused int64 + + if !rows.Next() { + if err := rows.Err(); err != nil { + return err + } + return errors.New("zero rows in result") + } + if err := rows.Scan( + &actualJob.id, &actualJob.status, &actualJob.fractionCompleted, &unused, &unused, &unused, + ); err != nil { + return err + } + if rows.Next() { + return errors.New("more than one row in result") + } + + sqlDB.QueryRow(t, + `SELECT job_id, status, fraction_completed FROM crdb_internal.jobs WHERE job_id = $1`, actualJob.id, + ).Scan( + &expectedJob.id, &expectedJob.status, &expectedJob.fractionCompleted, + ) + + if e, a := expectedJob, actualJob; !reflect.DeepEqual(e, a) { + return errors.Errorf("result does not match system.jobs:\n%s", + strings.Join(pretty.Diff(e, a), "\n")) + } + + return nil +} + +func backupRestoreTestSetupEmptyWithParams( + t testing.TB, + clusterSize int, + dir string, + init func(tc *testcluster.TestCluster), + params base.TestClusterArgs, +) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, cleanup func()) { + ctx := logtags.AddTag(context.Background(), "backup-restore-test-setup-empty", nil) + + params.ServerArgs.ExternalIODir = dir + if len(params.ServerArgsPerNode) > 0 { + for i := range params.ServerArgsPerNode { + param := params.ServerArgsPerNode[i] + param.ExternalIODir = dir + params.ServerArgsPerNode[i] = param + } + } + tc = testcluster.StartTestCluster(t, clusterSize, params) + init(tc) + + sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) + + cleanupFn := func() { + tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs + } + + return tc, sqlDB, cleanupFn +} + +func createEmptyCluster( + t testing.TB, clusterSize int, +) (sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { + ctx := context.Background() + + dir, dirCleanupFn := testutils.TempDir(t) + params := base.TestClusterArgs{} + params.ServerArgs.ExternalIODir = dir + tc := testcluster.StartTestCluster(t, clusterSize, params) + + sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) + + cleanupFn := func() { + tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs + dirCleanupFn() // cleans up dir, which is the nodelocal:// storage + } + + return sqlDB, dir, cleanupFn +} + +// getStatsQuery returns a SQL query that will return the properties of the +// statistics on a table that are expected to remain the same after being +// restored on a new cluster. +func getStatsQuery(tableName string) string { + return fmt.Sprintf(`SELECT + statistics_name, + column_names, + row_count, + distinct_count, + null_count + FROM [SHOW STATISTICS FOR TABLE %s]`, tableName) +} + +// injectStats directly injects some arbitrary statistic into a given table for +// a specified column. +// See injectStatsWithRowCount. +func injectStats( + t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, columnName string, +) [][]string { + return injectStatsWithRowCount(t, sqlDB, tableName, columnName, 100 /* rowCount */) +} + +// injectStatsWithRowCount directly injects some statistics specifying some row +// count for a column in the given table. +// N.B. This should be used in backup testing over CREATE STATISTICS since it +// ensures that the stats cache will be up to date during a subsequent BACKUP. +func injectStatsWithRowCount( + t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, columnName string, rowCount int, +) [][]string { + sqlDB.Exec(t, fmt.Sprintf(`ALTER TABLE %s INJECT STATISTICS '[ + { + "columns": ["%s"], + "created_at": "2018-01-01 1:00:00.00000+00:00", + "row_count": %d, + "distinct_count": %d + } + ]'`, tableName, columnName, rowCount, rowCount)) + return sqlDB.QueryStr(t, getStatsQuery(tableName)) +} + +func makeInsecureHTTPServer(t *testing.T) (string, func()) { + t.Helper() + + const badHeadResponse = "bad-head-response" + + tmp, dirCleanup := testutils.TempDir(t) + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + localfile := filepath.Join(tmp, filepath.Base(r.URL.Path)) + switch r.Method { + case "PUT": + f, err := os.Create(localfile) + if err != nil { + http.Error(w, err.Error(), 500) + return + } + defer f.Close() + if _, err := io.Copy(f, r.Body); err != nil { + http.Error(w, err.Error(), 500) + return + } + w.WriteHeader(201) + case "GET", "HEAD": + if filepath.Base(localfile) == badHeadResponse { + http.Error(w, "HEAD not implemented", 500) + return + } + http.ServeFile(w, r, localfile) + case "DELETE": + if err := os.Remove(localfile); err != nil { + http.Error(w, err.Error(), 500) + return + } + w.WriteHeader(204) + default: + http.Error(w, "unsupported method "+r.Method, 400) + } + })) + + cleanup := func() { + srv.Close() + dirCleanup() + } + + t.Logf("Mock HTTP Storage %q", srv.URL) + uri, err := url.Parse(srv.URL) + if err != nil { + srv.Close() + t.Fatal(err) + } + uri.Path = filepath.Join(uri.Path, "testing") + return uri.String(), cleanup +} + +// thresholdBlocker is a small wrapper around channels that are commonly used to +// block operations during testing. +// For example, it can be used in conjection with the RunBeforeBackfillChunk and +// BulkAdderFlushesEveryBatch cluster settings. The SQLSchemaChanger knob can be +// used to control the chunk size. +type thresholdBlocker struct { + threshold int + reachedThreshold chan struct{} + canProceed chan struct{} +} + +func (t thresholdBlocker) maybeBlock(count int) { + if count == t.threshold { + close(t.reachedThreshold) + <-t.canProceed + } +} + +func (t thresholdBlocker) waitUntilBlocked() { + <-t.reachedThreshold +} + +func (t thresholdBlocker) allowToProceed() { + close(t.canProceed) +} + +func makeThresholdBlocker(threshold int) thresholdBlocker { + return thresholdBlocker{ + threshold: threshold, + reachedThreshold: make(chan struct{}), + canProceed: make(chan struct{}), + } +} + +// getSpansFromManifest returns the spans that describe the data included in a +// given backup. +func getSpansFromManifest(ctx context.Context, t *testing.T, backupPath string) roachpb.Spans { + backupManifestBytes, err := ioutil.ReadFile(backupPath + "/" + backupManifestName) + require.NoError(t, err) + var backupManifest BackupManifest + decompressedBytes, err := decompressData(ctx, nil, backupManifestBytes) + require.NoError(t, err) + require.NoError(t, protoutil.Unmarshal(decompressedBytes, &backupManifest)) + spans := make([]roachpb.Span, 0, len(backupManifest.Files)) + for _, file := range backupManifest.Files { + spans = append(spans, file.Span) + } + mergedSpans, _ := roachpb.MergeSpans(&spans) + return mergedSpans +} + +func getKVCount(ctx context.Context, kvDB *kv.DB, dbName, tableName string) (int, error) { + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, tableName) + tablePrefix := keys.SystemSQLCodec.TablePrefix(uint32(tableDesc.GetID())) + tableEnd := tablePrefix.PrefixEnd() + kvs, err := kvDB.Scan(ctx, tablePrefix, tableEnd, 0) + return len(kvs), err +} + +// uriFmtStringAndArgs returns format strings like "$1" or "($1, $2, $3)" and +// an []interface{} of URIs for the BACKUP/RESTORE queries. +func uriFmtStringAndArgs(uris []string) (string, []interface{}) { + urisForFormat := make([]interface{}, len(uris)) + var fmtString strings.Builder + if len(uris) > 1 { + fmtString.WriteString("(") + } + for i, uri := range uris { + if i > 0 { + fmtString.WriteString(", ") + } + fmtString.WriteString(fmt.Sprintf("$%d", i+1)) + urisForFormat[i] = uri + } + if len(uris) > 1 { + fmtString.WriteString(")") + } + return fmtString.String(), urisForFormat +} + // CheckEmittedEvents is a helper method used by IMPORT and RESTORE tests to // ensure events are emitted deterministically. func CheckEmittedEvents( @@ -71,3 +453,90 @@ func CheckEmittedEvents( } var cmLogRe = regexp.MustCompile(`event_log\.go`) + +// waitForTableSplit waits for the dbName.tableName range to split. This is +// often used by tests that rely on SpanConfig fields being applied to the table +// span. +func waitForTableSplit(t *testing.T, conn *gosql.DB, tableName, dbName string) { + t.Helper() + testutils.SucceedsSoon(t, func() error { + count := 0 + if err := conn.QueryRow( + "SELECT count(*) "+ + "FROM crdb_internal.ranges_no_leases "+ + "WHERE table_name = $1 "+ + "AND database_name = $2", + tableName, dbName).Scan(&count); err != nil { + return err + } + if count == 0 { + return errors.New("waiting for table split") + } + return nil + }) +} + +func getTableStartKey(t *testing.T, conn *gosql.DB, tableName, dbName string) roachpb.Key { + t.Helper() + row := conn.QueryRow( + "SELECT start_key "+ + "FROM crdb_internal.ranges_no_leases "+ + "WHERE table_name = $1 "+ + "AND database_name = $2 "+ + "ORDER BY start_key ASC "+ + "LIMIT 1", + tableName, dbName) + var startKey roachpb.Key + require.NoError(t, row.Scan(&startKey)) + return startKey +} + +func getFirstStoreReplica( + t *testing.T, s serverutils.TestServerInterface, key roachpb.Key, +) (*kvserver.Store, *kvserver.Replica) { + t.Helper() + store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) + require.NoError(t, err) + var repl *kvserver.Replica + testutils.SucceedsSoon(t, func() error { + repl = store.LookupReplica(roachpb.RKey(key)) + if repl == nil { + return errors.New(`could not find replica`) + } + return nil + }) + return store, repl +} + +func getStoreAndReplica( + t *testing.T, tc *testcluster.TestCluster, conn *gosql.DB, tableName, dbName string, +) (*kvserver.Store, *kvserver.Replica) { + t.Helper() + startKey := getTableStartKey(t, conn, tableName, dbName) + // Okay great now we have a key and can go find replicas and stores and what not. + r := tc.LookupRangeOrFatal(t, startKey) + l, _, err := tc.FindRangeLease(r, nil) + require.NoError(t, err) + + lhServer := tc.Server(int(l.Replica.NodeID) - 1) + return getFirstStoreReplica(t, lhServer, startKey) +} + +// waitForReplicaFieldToBeSet can be used to wait for the replica corresponding +// to `dbName.tableName` to have a field set on it. +func waitForReplicaFieldToBeSet( + t *testing.T, + tc *testcluster.TestCluster, + conn *gosql.DB, + tableName, dbName string, + isReplicaFieldSet func(r *kvserver.Replica) (bool, error), +) { + t.Helper() + testutils.SucceedsSoon(t, func() error { + _, r := getStoreAndReplica(t, tc, conn, tableName, dbName) + if isSet, err := isReplicaFieldSet(r); !isSet { + return err + } + return nil + }) +} diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup new file mode 100644 index 000000000000..5e549f408456 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup @@ -0,0 +1,52 @@ +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(); +CREATE TABLE db.t2(); +---- + +query-sql +SELECT id FROM system.namespace WHERE name='t1' +---- +106 + +query-sql +SELECT id FROM system.namespace WHERE name='t2' +---- +107 + +# We only expect there to be span config entries for tables t1 and t2. +translate database=db +---- +/Table/10{6-7} range default +/Table/10{7-8} range default + +# Alter table t1 to mark its data ephemeral. +exec-sql +ALTER TABLE db.t1 SET (exclude_data_from_backup = true) +---- + +translate database=db +---- +/Table/10{6-7} exclude_data_from_backup=true +/Table/10{7-8} range default + +# Translating the tables in the database individually should result in the same +# config as above. + +translate database=db table=t1 +---- +/Table/10{6-7} exclude_data_from_backup=true + +translate database=db table=t2 +---- +/Table/10{7-8} range default + +# Alter table t1 to unmark its data ephemeral. +exec-sql +ALTER TABLE db.t1 SET (exclude_data_from_backup = false); +---- + +translate database=db +---- +/Table/10{6-7} range default +/Table/10{7-8} range default diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/exclude_data_from_backup b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/exclude_data_from_backup new file mode 100644 index 000000000000..f221ccb6cb88 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/exclude_data_from_backup @@ -0,0 +1,52 @@ +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(); +CREATE TABLE db.t2(); +---- + +query-sql +SELECT id FROM system.namespace WHERE name='t1' +---- +106 + +query-sql +SELECT id FROM system.namespace WHERE name='t2' +---- +107 + +# We only expect there to be span config entries for tables t1 and t2. +translate database=db +---- +/Tenant/10/Table/10{6-7} range default +/Tenant/10/Table/10{7-8} range default + +# Alter table t1 to mark its data ephemeral. +exec-sql +ALTER TABLE db.t1 SET (exclude_data_from_backup = true) +---- + +translate database=db +---- +/Tenant/10/Table/10{6-7} exclude_data_from_backup=true +/Tenant/10/Table/10{7-8} range default + +# Translating the tables in the database individually should result in the same +# config as above. + +translate database=db table=t1 +---- +/Tenant/10/Table/10{6-7} exclude_data_from_backup=true + +translate database=db table=t2 +---- +/Tenant/10/Table/10{7-8} range default + +# Alter table t1 to unmark its data ephemeral. +exec-sql +ALTER TABLE db.t1 SET (exclude_data_from_backup = false); +---- + +translate database=db +---- +/Tenant/10/Table/10{6-7} range default +/Tenant/10/Table/10{7-8} range default diff --git a/pkg/kv/kvserver/batcheval/cmd_export.go b/pkg/kv/kvserver/batcheval/cmd_export.go index 9753882ef1f4..3f58cf52220a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export.go +++ b/pkg/kv/kvserver/batcheval/cmd_export.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" @@ -112,6 +113,17 @@ func evalExport( } evalExportSpan.RecordStructured(&evalExportTrace) + // Table's marked to be excluded from backup are expected to be configured + // with a short GC TTL. Additionally, backup excludes such table's from being + // protected from GC when writing ProtectedTimestamp records. The + // ExportRequest is likely to find its target data has been GC'ed at this + // point, and so if the range being exported is part of such a table, we do + // not want to send back any row data to be backed up. + if cArgs.EvalCtx.ExcludeDataFromBackup() { + log.Infof(ctx, "[%s, %s) is part of a table excluded from backup, returning empty ExportResponse", args.Key, args.EndKey) + return result.Result{}, nil + } + if !args.ReturnSST { return result.Result{}, errors.New("ReturnSST is required") } diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 72dd27d4ffa7..212345adb615 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -98,6 +98,7 @@ type EvalContext interface { GetLastSplitQPS() float64 GetGCThreshold() hlc.Timestamp + ExcludeDataFromBackup() bool GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error) GetLease() (roachpb.Lease, roachpb.Lease) GetRangeInfo(context.Context) roachpb.RangeInfo @@ -231,6 +232,9 @@ func (m *mockEvalCtxImpl) CanCreateTxnRecord( func (m *mockEvalCtxImpl) GetGCThreshold() hlc.Timestamp { return m.GCThreshold } +func (m *mockEvalCtxImpl) ExcludeDataFromBackup() bool { + return false +} func (m *mockEvalCtxImpl) GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error) { panic("unimplemented") } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 6aa0d1612d59..9577d00d2ffc 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -911,6 +911,18 @@ func (r *Replica) GetGCThreshold() hlc.Timestamp { return *r.mu.state.GCThreshold } +// ExcludeDataFromBackup returns whether the replica is to be excluded from a +// backup. +func (r *Replica) ExcludeDataFromBackup() bool { + r.mu.RLock() + defer r.mu.RUnlock() + return r.mu.conf.ExcludeDataFromBackup +} + +func (r *Replica) exludeReplicaFromBackupRLocked() bool { + return r.mu.conf.ExcludeDataFromBackup +} + // Version returns the replica version. func (r *Replica) Version() roachpb.Version { if r.mu.state.Version == nil { @@ -1514,8 +1526,9 @@ func (r *Replica) checkTSAboveGCThresholdRLocked( return nil } return &roachpb.BatchTimestampBeforeGCError{ - Timestamp: ts, - Threshold: threshold, + Timestamp: ts, + Threshold: threshold, + DataExcludedFromBackup: r.exludeReplicaFromBackupRLocked(), } } diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 5e197e16ad8a..e39b42c1fe0a 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -165,6 +165,12 @@ func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp { return rec.i.GetGCThreshold() } +// ExcludeDataFromBackup returns whether the replica is to be excluded from a +// backup. +func (rec SpanSetReplicaEvalContext) ExcludeDataFromBackup() bool { + return rec.i.ExcludeDataFromBackup() +} + // String implements Stringer. func (rec SpanSetReplicaEvalContext) String() string { return rec.i.String() diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 5e46b6c79962..bca737cad372 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -484,6 +484,10 @@ message IntegerOverflowError { message BatchTimestampBeforeGCError { optional util.hlc.Timestamp Timestamp = 1 [(gogoproto.nullable) = false]; optional util.hlc.Timestamp Threshold = 2 [(gogoproto.nullable) = false]; + // DataExcludedFromBackup is set to true if the request is targeting a range + // that has been marked as excluded from a backup via + // `ALTER TABLE ... SET (exclude_data_from_backup = true)`. + optional bool data_excluded_from_backup = 3 [(gogoproto.nullable) = false]; } // A MVCCHistoryMutationError indicates that MVCC history was unexpectedly diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index b4664ba29b73..857e87c757b0 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -164,6 +164,14 @@ message SpanConfig { // RangefeedEnabled determines whether rangefeeds are enabled over the // specific range. bool rangefeed_enabled = 10; + + // ExcludeDataFromBackup specifies if the range has been marked to be excluded + // from a backup targeting the table represented by this keyspace. This + // information is used when ExportRequests issued by backups are being + // serviced in KV, to decide whether or not to send back any row data. + bool exclude_data_from_backup = 11; + + // Next ID: 12 } // SpanConfigEntry ties a span to its corresponding config. diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 70943d8704d6..399372d90cd9 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -278,6 +278,10 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( ptsStateReader.getProtectionPoliciesForSchemaObject(desc.GetID()), ptsStateReader.getProtectionPoliciesForSchemaObject(desc.GetParentID())...) + // Set whether the table's row data has been marked to be excluded from + // backups. + tableSpanConfig.ExcludeDataFromBackup = desc.(catalog.TableDescriptor).GetExcludeDataFromBackup() + records := make([]spanconfig.Record, 0) if desc.GetID() == keys.DescriptorTableID { // We have some special handling for `system.descriptor` on account of @@ -379,9 +383,10 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( // Add an entry for the subzone. subzoneSpanConfig := zone.Subzones[zone.SubzoneSpans[i].SubzoneIndex].Config.AsSpanConfig() - // Copy the ProtectionPolicies that apply to the table's SpanConfig onto its + // Copy relevant fields that apply to the table's SpanConfig onto its // SubzoneSpanConfig. subzoneSpanConfig.GCPolicy.ProtectionPolicies = tableSpanConfig.GCPolicy.ProtectionPolicies[:] + subzoneSpanConfig.ExcludeDataFromBackup = tableSpanConfig.ExcludeDataFromBackup if isSystemDesc { // same as above subzoneSpanConfig.RangefeedEnabled = true subzoneSpanConfig.GCPolicy.IgnoreStrictEnforcement = true diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 5f5be7b6a5db..fa452e099182 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -279,6 +279,9 @@ func PrintSpanConfigDiffedAgainstDefaults(conf roachpb.SpanConfig) string { } diffs = append(diffs, fmt.Sprintf("pts=[%s]", strings.Join(timestamps, " "))) } + if conf.ExcludeDataFromBackup != defaultConf.ExcludeDataFromBackup { + diffs = append(diffs, fmt.Sprintf("exclude_data_from_backup=%v", conf.ExcludeDataFromBackup)) + } return strings.Join(diffs, " ") }